Merge branch 'ClickHouse:master' into fix_full_text_with_multi_col

This commit is contained in:
siyuan 2024-08-28 10:18:44 +08:00 committed by GitHub
commit 78f17d856b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
94 changed files with 1394 additions and 594 deletions

View File

@ -18,7 +18,9 @@
#define Net_HTTPResponse_INCLUDED
#include <map>
#include <vector>
#include "Poco/Net/HTTPCookie.h"
#include "Poco/Net/HTTPMessage.h"
#include "Poco/Net/Net.h"
@ -180,6 +182,8 @@ namespace Net
/// May throw an exception in case of a malformed
/// Set-Cookie header.
void getHeaders(std::map<std::string, std::string> & headers) const;
void write(std::ostream & ostr) const;
/// Writes the HTTP response to the given
/// output stream.

View File

@ -209,6 +209,15 @@ void HTTPResponse::getCookies(std::vector<HTTPCookie>& cookies) const
}
}
void HTTPResponse::getHeaders(std::map<std::string, std::string> & headers) const
{
headers.clear();
for (const auto & it : *this)
{
headers.emplace(it.first, it.second);
}
}
void HTTPResponse::write(std::ostream& ostr) const
{

View File

@ -6,28 +6,34 @@ sidebar_label: Iceberg
# Iceberg Table Engine
This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3.
This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure and locally stored tables.
## Create Table
Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table.
Note that the Iceberg table must already exist in the storage, this command does not take DDL parameters to create a new table.
``` sql
CREATE TABLE iceberg_table
ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,])
CREATE TABLE iceberg_table_s3
ENGINE = IcebergS3(url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression])
CREATE TABLE iceberg_table_azure
ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])
CREATE TABLE iceberg_table_local
ENGINE = IcebergLocal(path_to_table, [,format] [,compression_method])
```
**Engine parameters**
**Engine arguments**
- `url` — url with the path to an existing Iceberg table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Description of the arguments coincides with description of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly.
`format` stands for the format of data files in the Iceberg table.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
**Example**
```sql
CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
CREATE TABLE iceberg_table ENGINE=IcebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
```
Using named collections:
@ -45,9 +51,15 @@ Using named collections:
```
```sql
CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table')
CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table')
```
**Aliases**
Table engine `Iceberg` is an alias to `IcebergS3` now.
## See also
- [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md)

View File

@ -109,6 +109,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da
- `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`.
- `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`.
## Storage Settings {#storage-settings}

View File

@ -5633,7 +5633,6 @@ Default value: `1GiB`.
## use_json_alias_for_old_object_type
When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type.
This setting requires server restart to take effect when changed.
Default value: `false`.

View File

@ -4287,7 +4287,7 @@ Result:
## fromModifiedJulianDay
Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number to a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD`. This function supports day number from `-678941` to `2973119` (which represent 0000-01-01 and 9999-12-31 respectively). It raises an exception if the day number is outside of the supported range.
Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number to a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD`. This function supports day number from `-678941` to `2973483` (which represent 0000-01-01 and 9999-12-31 respectively). It raises an exception if the day number is outside of the supported range.
**Syntax**

View File

@ -6,35 +6,37 @@ sidebar_label: iceberg
# iceberg Table Function
Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3.
Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure or locally stored.
## Syntax
``` sql
iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure])
icebergS3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method])
icebergS3(named_collection[, option=value [,..]])
icebergAzure(connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method])
icebergAzure(named_collection[, option=value [,..]])
icebergLocal(path_to_table, [,format] [,compression_method])
icebergLocal(named_collection[, option=value [,..]])
```
## Arguments
- `url` — Bucket url with the path to an existing Iceberg table in S3.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3).
- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used.
- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`.
Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md).
Description of the arguments coincides with description of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly.
`format` stands for the format of data files in the Iceberg table.
**Returned value**
A table with the specified structure for reading data in the specified Iceberg table in S3.
A table with the specified structure for reading data in the specified Iceberg table.
**Example**
```sql
SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
```
:::important
ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine.
ClickHouse currently supports reading v1 and v2 of the Iceberg format via the `icebergS3`, `icebergAzure` and `icebergLocal` table functions and `IcebergS3`, `icebergAzure` ans `icebergLocal` table engines.
:::
## Defining a named collection
@ -56,10 +58,14 @@ Here is an example of configuring a named collection for storing the URL and cre
```
```sql
SELECT * FROM iceberg(iceberg_conf, filename = 'test_table')
DESCRIBE iceberg(iceberg_conf, filename = 'test_table')
SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table')
DESCRIBE icebergS3(iceberg_conf, filename = 'test_table')
```
**Aliases**
Table function `iceberg` is an alias to `icebergS3` now.
**See Also**
- [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md)

View File

@ -54,6 +54,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it
- `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`.
- `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`.
## Hive-style partitioning {#hive-style-partitioning}

View File

@ -1157,7 +1157,7 @@ SELECT toModifiedJulianDayOrNull('2020-01-01');
## fromModifiedJulianDay {#frommodifiedjulianday}
将 [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) 数字转换为 `YYYY-MM-DD` 文本格式的 [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) 日期。该函数支持从 `-678941``2973119` 的天数(分别代表 0000-01-01 和 9999-12-31。如果天数超出支持范围则会引发异常。
将 [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) 数字转换为 `YYYY-MM-DD` 文本格式的 [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) 日期。该函数支持从 `-678941``2973483` 的天数(分别代表 0000-01-01 和 9999-12-31。如果天数超出支持范围则会引发异常。
**语法**

View File

@ -978,6 +978,7 @@ try
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
* At this moment, no one could own shared part of Context.
*/
global_context->resetSharedContext();
global_context.reset();
shared_context.reset();
LOG_DEBUG(log, "Destroyed global context.");

View File

@ -100,6 +100,7 @@ protected:
auto buf = BuilderRWBufferFromHTTP(getPingURI())
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withTimeouts(getHTTPTimeouts())
.withSettings(getContext()->getReadSettings())
.create(credentials);
return checkString(PING_OK_ANSWER, *buf);
@ -206,6 +207,7 @@ protected:
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(getHTTPTimeouts())
.withSettings(getContext()->getReadSettings())
.create(credentials);
bool res = false;
@ -232,6 +234,7 @@ protected:
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(getHTTPTimeouts())
.withSettings(getContext()->getReadSettings())
.create(credentials);
std::string character;

View File

@ -111,6 +111,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage)
add_headers_and_sources(dbms Storages/ObjectStorage/Azure)
add_headers_and_sources(dbms Storages/ObjectStorage/S3)
add_headers_and_sources(dbms Storages/ObjectStorage/HDFS)
add_headers_and_sources(dbms Storages/ObjectStorage/Local)
add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes)
add_headers_and_sources(dbms Common/NamedCollections)

View File

@ -22,7 +22,6 @@
#include <cstring>
#include <unistd.h>
#include <algorithm>
#include <typeinfo>
#include <iostream>
#include <memory>

View File

@ -1,10 +1,12 @@
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeObjectDeprecated.h>
#include <DataTypes/Serializations/SerializationJSON.h>
#include <DataTypes/Serializations/SerializationObjectTypedPath.h>
#include <DataTypes/Serializations/SerializationObjectDynamicPath.h>
#include <DataTypes/Serializations/SerializationSubObject.h>
#include <Columns/ColumnObject.h>
#include <Common/CurrentThread.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
@ -513,13 +515,24 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject::
static DataTypePtr createJSON(const ASTPtr & arguments)
{
auto context = CurrentThread::getQueryContext();
if (!context)
context = Context::getGlobalContextInstance();
if (context->getSettingsRef().use_json_alias_for_old_object_type)
{
if (arguments && !arguments->children.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set setting allow_experimental_json_type = 1");
return std::make_shared<DataTypeObjectDeprecated>("JSON", false);
}
return createObject(arguments, DataTypeObject::SchemaFormat::JSON);
}
void registerDataTypeJSON(DataTypeFactory & factory)
{
if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type)
factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive);
factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive);
}
}

View File

@ -78,10 +78,6 @@ static DataTypePtr create(const ASTPtr & arguments)
void registerDataTypeObjectDeprecated(DataTypeFactory & factory)
{
factory.registerDataType("Object", create);
if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type)
factory.registerSimpleDataType("JSON",
[] { return std::make_shared<DataTypeObjectDeprecated>("JSON", false); },
DataTypeFactory::Case::Insensitive);
}
}

View File

@ -43,39 +43,21 @@ bool LocalObjectStorage::exists(const StoredObject & object) const
std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObjects( /// NOLINT
const StoredObjects & objects,
const ReadSettings & read_settings,
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
std::optional<size_t>,
std::optional<size_t>) const
{
auto modified_settings = patchSettings(read_settings);
auto global_context = Context::getGlobalContextInstance();
auto read_buffer_creator =
[=] (bool /* restricted_seek */, const StoredObject & object)
-> std::unique_ptr<ReadBufferFromFileBase>
{
return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size);
};
auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr<ReadBufferFromFileBase>
{ return std::make_unique<ReadBufferFromFile>(object.remote_path); };
switch (read_settings.remote_fs_method)
{
case RemoteFSReadMethod::read:
{
return std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator), objects, "file:", modified_settings,
global_context->getFilesystemCacheLog(), /* use_external_buffer */false);
}
case RemoteFSReadMethod::threadpool:
{
auto impl = std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator), objects, "file:", modified_settings,
global_context->getFilesystemCacheLog(), /* use_external_buffer */true);
auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
return std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(impl), reader, read_settings,
global_context->getAsyncReadCounters(),
global_context->getFilesystemReadPrefetchesLog());
}
}
return std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator),
objects,
"file:",
modified_settings,
global_context->getFilesystemCacheLog(),
/* use_external_buffer */ false);
}
ReadSettings LocalObjectStorage::patchSettings(const ReadSettings & read_settings) const

View File

@ -284,12 +284,12 @@ void OrdinalDate::init(int64_t modified_julian_day)
bool OrdinalDate::tryInit(int64_t modified_julian_day)
{
/// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively).
/// This function supports day number from -678941 to 2973483 (which represent 0000-01-01 and 9999-12-31 respectively).
if (modified_julian_day < -678941)
return false;
if (modified_julian_day > 2973119)
if (modified_julian_day > 2973483)
return false;
const auto a = modified_julian_day + 678575;

View File

@ -443,6 +443,7 @@ std::unique_ptr<ReadBuffer> ReadWriteBufferFromHTTP::initialize()
}
response.getCookies(cookies);
response.getHeaders(response_headers);
content_encoding = response.get("Content-Encoding", "");
// Remember file size. It'll be used to report eof in next nextImpl() call.
@ -680,6 +681,19 @@ std::string ReadWriteBufferFromHTTP::getResponseCookie(const std::string & name,
return def;
}
Map ReadWriteBufferFromHTTP::getResponseHeaders() const
{
Map map;
for (const auto & header : response_headers)
{
Tuple elem;
elem.emplace_back(header.first);
elem.emplace_back(header.second);
map.emplace_back(elem);
}
return map;
}
void ReadWriteBufferFromHTTP::setNextCallback(NextCallback next_callback_)
{
next_callback = next_callback_;

View File

@ -90,6 +90,9 @@ private:
std::unique_ptr<ReadBuffer> impl;
std::vector<Poco::Net::HTTPCookie> cookies;
std::map<String, String> response_headers;
HTTPHeaderEntries http_header_entries;
std::function<void(size_t)> next_callback;
@ -187,6 +190,8 @@ public:
HTTPFileInfo getFileInfo();
static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin);
Map getResponseHeaders() const;
};
using ReadWriteBufferFromHTTPPtr = std::unique_ptr<ReadWriteBufferFromHTTP>;

View File

@ -389,6 +389,10 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr
if (data_kind == DataKind::Preprocessed)
insert_query.format = "Native";
/// Query parameters make sense only for format Values.
if (insert_query.format == "Values")
entry->query_parameters = query_context->getQueryParameters();
InsertQuery key{query, query_context->getUserID(), query_context->getCurrentRoles(), settings, data_kind};
InsertDataPtr data_to_process;
std::future<void> insert_future;
@ -999,6 +1003,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
"Expected entry with data kind Parsed. Got: {}", entry->chunk.getDataKind());
auto buffer = std::make_unique<ReadBufferFromString>(*bytes);
executor.setQueryParameters(entry->query_parameters);
size_t num_bytes = bytes->size();
size_t num_rows = executor.execute(*buffer);

View File

@ -147,6 +147,7 @@ private:
const String format;
MemoryTracker * const user_memory_tracker;
const std::chrono::time_point<std::chrono::system_clock> create_time;
NameToNameMap query_parameters;
Entry(
DataChunk && chunk_,

View File

@ -893,6 +893,12 @@ ContextData::ContextData(const ContextData &o) :
{
}
void ContextData::resetSharedContext()
{
std::lock_guard<std::mutex> lock(mutex_shared_context);
shared = nullptr;
}
Context::Context() = default;
Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this<Context>(rhs) {}
@ -914,14 +920,6 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part)
return res;
}
void Context::initGlobal()
{
assert(!global_context_instance);
global_context_instance = shared_from_this();
DatabaseCatalog::init(shared_from_this());
EventNotifier::init();
}
SharedContextHolder Context::createShared()
{
return SharedContextHolder(std::make_unique<ContextSharedPart>());
@ -2692,7 +2690,11 @@ void Context::makeSessionContext()
void Context::makeGlobalContext()
{
initGlobal();
assert(!global_context_instance);
global_context_instance = shared_from_this();
DatabaseCatalog::init(shared_from_this());
EventNotifier::init();
global_context = shared_from_this();
}
@ -4088,8 +4090,13 @@ void Context::initializeTraceCollector()
}
/// Call after unexpected crash happen.
void Context::handleCrash() const TSA_NO_THREAD_SAFETY_ANALYSIS
void Context::handleCrash() const
{
std::lock_guard<std::mutex> lock(mutex_shared_context);
if (!shared)
return;
SharedLockGuard lock2(shared->mutex);
if (shared->system_logs)
shared->system_logs->handleCrash();
}

View File

@ -492,6 +492,8 @@ public:
KitchenSink kitchen_sink;
void resetSharedContext();
protected:
using SampleBlockCache = std::unordered_map<std::string, Block>;
mutable SampleBlockCache sample_block_cache;
@ -529,6 +531,10 @@ protected:
mutable ThrottlerPtr local_write_query_throttler; /// A query-wide throttler for local IO writes
mutable ThrottlerPtr backups_query_throttler; /// A query-wide throttler for BACKUPs
mutable std::mutex mutex_shared_context; /// mutex to avoid accessing destroyed shared context pointer
/// some Context methods can be called after the shared context is destroyed
/// example, Context::handleCrash() method - called from signal handler
};
/** A set of known objects that can be used in the query.
@ -1387,8 +1393,6 @@ private:
ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard<std::mutex> & lock);
void initGlobal();
void setUserID(const UUID & user_id_);
void setCurrentRolesImpl(const std::vector<UUID> & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr<const User> & user);

View File

@ -1,5 +1,6 @@
#include <Processors/Executors/StreamingFormatExecutor.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
namespace DB
{
@ -32,6 +33,13 @@ MutableColumns StreamingFormatExecutor::getResultColumns()
return ret_columns;
}
void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters)
{
/// Query parameters make sense only for format Values.
if (auto * values_format = typeid_cast<ValuesBlockInputFormat *>(format.get()))
values_format->setQueryParameters(parameters);
}
size_t StreamingFormatExecutor::execute(ReadBuffer & buffer)
{
format->setReadBuffer(buffer);

View File

@ -39,6 +39,9 @@ public:
/// Releases currently accumulated columns.
MutableColumns getResultColumns();
/// Sets query parameters for input format if applicable.
void setQueryParameters(const NameToNameMap & parameters);
private:
const Block header;
const InputFormatPtr format;

View File

@ -663,6 +663,16 @@ void ValuesBlockInputFormat::resetReadBuffer()
IInputFormat::resetReadBuffer();
}
void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters)
{
if (parameters == context->getQueryParameters())
return;
auto context_copy = Context::createCopy(context);
context_copy->setQueryParameters(parameters);
context = std::move(context_copy);
}
ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(buf, format_settings_), buf(in_)
{

View File

@ -38,6 +38,7 @@ public:
/// TODO: remove context somehow.
void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); }
void setQueryParameters(const NameToNameMap & parameters);
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }

View File

@ -807,7 +807,7 @@ MergeTreeDataPartBuilder IMergeTreeDataPart::getProjectionPartBuilder(const Stri
const char * projection_extension = is_temp_projection ? ".tmp_proj" : ".proj";
auto projection_storage = getDataPartStorage().getProjection(projection_name + projection_extension, !is_temp_projection);
MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage);
return builder.withPartInfo({"all", 0, 0, 0}).withParentPart(this);
return builder.withPartInfo(MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION).withParentPart(this);
}
void IMergeTreeDataPart::addProjectionPart(
@ -1334,17 +1334,6 @@ void IMergeTreeDataPart::loadRowsCount()
auto buf = metadata_manager->read("count.txt");
readIntText(rows_count, *buf);
assertEOF(*buf);
if (!index_granularity.empty() && rows_count < index_granularity.getTotalRows() && index_granularity_info.fixed_index_granularity)
{
/// Adjust last granule size to match the number of rows in the part in case of fixed index_granularity.
index_granularity.popMark();
index_granularity.appendMark(rows_count % index_granularity_info.fixed_index_granularity);
if (rows_count != index_granularity.getTotalRows())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Index granularity total rows in part {} does not match rows_count: {}, instead of {}",
name, index_granularity.getTotalRows(), rows_count);
}
};
if (index_granularity.empty())

View File

@ -6,10 +6,18 @@
#include <Common/CurrentThread.h>
#include <Common/MemoryTracker.h>
#include <Common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
const MergeTreePartInfo MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION = {"all", 0, 0, 0};
MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context)
: table_id{table_id_}
, partition_id{future_part->part_info.partition_id}
@ -21,8 +29,23 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta
, merge_type{future_part->merge_type}
, merge_algorithm{MergeAlgorithm::Undecided}
{
auto format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
if (result_part_name != result_part_info.getPartNameV1())
format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION;
/// FIXME why do we need a merge list element for projection parts at all?
bool is_fake_projection_part = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION;
size_t normal_parts_count = 0;
for (const auto & source_part : future_part->parts)
{
if (!is_fake_projection_part && !source_part->getParentPart())
{
++normal_parts_count;
if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version)))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1());
}
source_part_names.emplace_back(source_part->name);
source_part_paths.emplace_back(source_part->getDataPartStorage().getFullPath());
@ -35,13 +58,17 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta
if (!future_part->parts.empty())
{
source_data_version = future_part->parts[0]->info.getDataVersion();
is_mutation = (result_part_info.getDataVersion() != source_data_version);
is_mutation = (result_part_info.level == future_part->parts[0]->info.level) && !is_fake_projection_part;
WriteBufferFromString out(partition);
const auto & part = future_part->parts[0];
part->partition.serializeText(part->storage, out, {});
}
if (!is_fake_projection_part && is_mutation && normal_parts_count != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}: {}", future_part->parts.size(),
result_part_info.getPartNameV1(), fmt::join(source_part_names, ", "));
thread_group = ThreadGroup::createForBackgroundProcess(context);
}

View File

@ -66,6 +66,8 @@ struct Settings;
struct MergeListElement : boost::noncopyable
{
static const MergeTreePartInfo FAKE_RESULT_PART_FOR_PROJECTION;
const StorageID table_id;
std::string partition_id;
std::string partition;

View File

@ -1021,7 +1021,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c
// TODO (ab): path in future_part is only for merge process introspection, which is not available for merges of projection parts.
// Let's comment this out to avoid code inconsistency and add it back after we implement projection merge introspection.
// projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/";
projection_future_part->part_info = {"all", 0, 0, 0};
projection_future_part->part_info = MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION;
MergeTreeData::MergingParams projection_merging_params;
projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary;

View File

@ -577,10 +577,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
if (index_granularity_rows != index_granularity.getMarkRows(mark_num))
{
/// With fixed granularity we can have last mark with less rows than granularity
const bool is_last_mark = (mark_num + 1 == index_granularity.getMarksCount());
if (!index_granularity_info.fixed_index_granularity || !is_last_mark)
throw Exception(
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Incorrect mark rows for part {} for mark #{}"
" (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}",
@ -844,14 +841,7 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_
/// Without offset
rows_written_in_last_mark = 0;
}
if (compute_granularity)
{
index_granularity.popMark();
index_granularity.appendMark(new_rows_in_last_mark);
}
}
}
}

View File

@ -92,13 +92,14 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
/// We will find out how many rows we would have read without sampling.
LOG_DEBUG(log, "Preliminary index scan with condition: {}", key_condition.toString());
MarkRanges exact_ranges;
for (const auto & part : parts)
{
MarkRanges exact_ranges;
markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log);
for (const auto & range : exact_ranges)
MarkRanges part_ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log);
for (const auto & range : part_ranges)
rows_count += part->index_granularity.getRowsCountInRange(range);
}
UNUSED(exact_ranges);
return rows_count;
}

View File

@ -148,10 +148,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
{
if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7))
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage AzureBlobStorage requires 3 to 7 arguments: "
"AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, "
"[account_name, account_key, format, compression, structure)])");
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage AzureBlobStorage requires 3 to {} arguments: "
"AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, "
"[account_name, account_key, format, compression, structure)])",
(with_structure ? 8 : 7));
}
for (auto & engine_arg : engine_args)

View File

@ -3,7 +3,7 @@
#include "config.h"
#include <set>
#if USE_AWS_S3 && USE_PARQUET
#if USE_PARQUET
#include <Common/logger_useful.h>
#include <Columns/ColumnString.h>

View File

@ -2,7 +2,7 @@
#include "config.h"
#if USE_AWS_S3 && USE_AVRO
#if USE_AVRO
#include <Formats/FormatFactory.h>
#include <Storages/IStorage.h>

View File

@ -1,6 +1,6 @@
#include "config.h"
#if USE_AWS_S3 && USE_AVRO
#if USE_AVRO
#include <Common/logger_useful.h>
#include <Core/Settings.h>

View File

@ -1,6 +1,6 @@
#pragma once
#if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format.
#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format.
#include <Interpreters/Context_fwd.h>
#include <Core/Types.h>

View File

@ -2,10 +2,12 @@
#if USE_AWS_S3
#include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
#include <Storages/ObjectStorage/DataLakes/IStorageDataLake.h>
#include <Storages/ObjectStorage/DataLakes/IcebergMetadata.h>
#include <Storages/ObjectStorage/S3/Configuration.h>
# include <Storages/ObjectStorage/Azure/Configuration.h>
# include <Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h>
# include <Storages/ObjectStorage/DataLakes/IStorageDataLake.h>
# include <Storages/ObjectStorage/DataLakes/IcebergMetadata.h>
# include <Storages/ObjectStorage/Local/Configuration.h>
# include <Storages/ObjectStorage/S3/Configuration.h>
namespace DB
@ -22,6 +24,54 @@ void registerStorageIceberg(StorageFactory & factory)
auto configuration = std::make_shared<StorageS3Configuration>();
StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false);
return StorageIceberg::create(
configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode);
},
{
.supports_settings = false,
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
});
factory.registerStorage(
"IcebergS3",
[&](const StorageFactory::Arguments & args)
{
auto configuration = std::make_shared<StorageS3Configuration>();
StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false);
return StorageIceberg::create(
configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode);
},
{
.supports_settings = false,
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
});
factory.registerStorage(
"IcebergAzure",
[&](const StorageFactory::Arguments & args)
{
auto configuration = std::make_shared<StorageAzureConfiguration>();
StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true);
return StorageIceberg::create(
configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode);
},
{
.supports_settings = false,
.supports_schema_inference = true,
.source_access_type = AccessType::AZURE,
});
factory.registerStorage(
"IcebergLocal",
[&](const StorageFactory::Arguments & args)
{
auto configuration = std::make_shared<StorageLocalConfiguration>();
StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false);
return StorageIceberg::create(
configuration, args.getContext(), args.table_id, args.columns,
args.constraints, args.comment, std::nullopt, args.mode);
@ -29,7 +79,7 @@ void registerStorageIceberg(StorageFactory & factory)
{
.supports_settings = false,
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
.source_access_type = AccessType::FILE,
});
}

View File

@ -0,0 +1,77 @@
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Storages/ObjectStorage/Local/Configuration.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include "Common/NamedCollections/NamedCollections.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr)
{
path = collection.get<String>("path");
format = collection.getOrDefault<String>("format", "auto");
compression_method = collection.getOrDefault<String>("compression_method", collection.getOrDefault<String>("compression", "auto"));
structure = collection.getOrDefault<String>("structure", "auto");
paths = {path};
}
void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure)
{
const size_t max_args_num = with_structure ? 4 : 3;
if (args.empty() || args.size() > max_args_num)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num);
}
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
path = checkAndGetLiteralArgument<String>(args[0], "path");
if (args.size() > 1)
{
format = checkAndGetLiteralArgument<String>(args[1], "format_name");
}
if (with_structure)
{
if (args.size() > 2)
{
structure = checkAndGetLiteralArgument<String>(args[2], "structure");
}
if (args.size() > 3)
{
compression_method = checkAndGetLiteralArgument<String>(args[3], "compression_method");
}
}
else if (args.size() > 2)
{
compression_method = checkAndGetLiteralArgument<String>(args[2], "compression_method");
}
paths = {path};
}
StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const
{
const auto & settings = context->getSettingsRef();
return StorageObjectStorage::QuerySettings{
.truncate_on_insert = settings.engine_file_truncate_on_insert,
.create_new_file_on_insert = false,
.schema_inference_use_cache = settings.schema_inference_use_cache_for_file,
.schema_inference_mode = settings.schema_inference_mode,
.skip_empty_files = settings.engine_file_skip_empty_files,
.list_object_keys_size = 0,
.throw_on_zero_files_match = false,
.ignore_non_existent_file = false};
}
}

View File

@ -0,0 +1,52 @@
#pragma once
#include <memory>
#include "Disks/ObjectStorages/Local/LocalObjectStorage.h"
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
class StorageLocalConfiguration : public StorageObjectStorage::Configuration
{
public:
using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr;
static constexpr auto type_name = "local";
StorageLocalConfiguration() = default;
StorageLocalConfiguration(const StorageLocalConfiguration & other) = default;
std::string getTypeName() const override { return type_name; }
std::string getEngineName() const override { return "Local"; }
Path getPath() const override { return path; }
void setPath(const Path & path_) override { path = path_; }
const Paths & getPaths() const override { return paths; }
void setPaths(const Paths & paths_) override { paths = paths_; }
String getNamespace() const override { return ""; }
String getDataSourceDescription() const override { return ""; }
StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override;
ConfigurationPtr clone() override { return std::make_shared<StorageLocalConfiguration>(*this); }
ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared<LocalObjectStorage>("/"); }
void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { }
private:
void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;
void fromAST(ASTs & args, ContextPtr context, bool with_structure) override;
Path path;
Paths paths;
};
}

View File

@ -465,6 +465,12 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c
DEFAULT_SCHEMA_CACHE_ELEMENTS));
return schema_cache;
}
else if (storage_type_name == "local")
{
static SchemaCache schema_cache(
context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_local", DEFAULT_SCHEMA_CACHE_ELEMENTS));
return schema_cache;
}
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name);
}

View File

@ -162,7 +162,7 @@ public:
ContextPtr local_context,
bool with_table_structure);
/// Storage type: s3, hdfs, azure.
/// Storage type: s3, hdfs, azure, local.
virtual std::string getTypeName() const = 0;
/// Engine name: S3, HDFS, Azure.
virtual std::string getEngineName() const = 0;

View File

@ -417,10 +417,7 @@ std::future<StorageObjectStorageSource::ReaderHolder> StorageObjectStorageSource
}
std::unique_ptr<ReadBuffer> StorageObjectStorageSource::createReadBuffer(
const ObjectInfo & object_info,
const ObjectStoragePtr & object_storage,
const ContextPtr & context_,
const LoggerPtr & log)
const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log)
{
const auto & object_size = object_info.metadata->size_bytes;

View File

@ -44,10 +44,11 @@
#include <IO/HTTPHeaderEntries.h>
#include <algorithm>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeString.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Poco/Net/HTTPRequest.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace ProfileEvents
{
@ -166,7 +167,19 @@ IStorageURLBase::IStorageURLBase(
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context_, getSampleURI(uri, context_), format_settings));
auto virtual_columns_desc = VirtualColumnUtils::getVirtualsForFileLikeStorage(
storage_metadata.columns, context_, getSampleURI(uri, context_), format_settings);
if (!storage_metadata.getColumns().has("_headers"))
{
virtual_columns_desc.addEphemeral(
"_headers",
std::make_shared<DataTypeMap>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())),
"");
}
setVirtuals(virtual_columns_desc);
setInMemoryMetadata(storage_metadata);
}
@ -292,11 +305,13 @@ StorageURLSource::StorageURLSource(
const URIParams & params,
bool glob_url,
bool need_only_count_)
: SourceWithKeyCondition(info.source_header, false), WithContext(context_)
: SourceWithKeyCondition(info.source_header, false)
, WithContext(context_)
, name(std::move(name_))
, columns_description(info.columns_description)
, requested_columns(info.requested_columns)
, requested_virtual_columns(info.requested_virtual_columns)
, need_headers_virtual_column(info.requested_virtual_columns.contains("_headers"))
, requested_virtual_columns(info.requested_virtual_columns.eraseNames({"_headers"}))
, block_for_format(info.format_header)
, uri_iterator(uri_iterator_)
, format(format_)
@ -431,11 +446,28 @@ Chunk StorageURLSource::generate()
progress(num_rows, chunk_size ? chunk_size : chunk.bytes());
VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk(
chunk, requested_virtual_columns,
chunk,
requested_virtual_columns,
{
.path = curr_uri.getPath(),
.size = current_file_size,
}, getContext());
},
getContext());
chassert(dynamic_cast<ReadWriteBufferFromHTTP *>(read_buf.get()));
if (need_headers_virtual_column)
{
if (!http_response_headers_initialized)
{
http_response_headers = dynamic_cast<ReadWriteBufferFromHTTP *>(read_buf.get())->getResponseHeaders();
http_response_headers_initialized = true;
}
auto type = std::make_shared<DataTypeMap>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()));
chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)->convertToFullColumnIfConst());
}
return chunk;
}
@ -446,6 +478,7 @@ Chunk StorageURLSource::generate()
reader.reset();
input_format.reset();
read_buf.reset();
http_response_headers_initialized = false;
total_rows_in_file = 0;
}
return {};

View File

@ -220,6 +220,7 @@ private:
String name;
ColumnsDescription columns_description;
NamesAndTypesList requested_columns;
bool need_headers_virtual_column;
NamesAndTypesList requested_virtual_columns;
Block block_for_format;
std::shared_ptr<IteratorWrapper> uri_iterator;
@ -233,6 +234,9 @@ private:
Poco::Net::HTTPBasicCredentials credentials;
Map http_response_headers;
bool http_response_headers_initialized = false;
std::unique_ptr<ReadBuffer> read_buf;
std::shared_ptr<IInputFormat> input_format;
std::unique_ptr<QueryPipeline> pipeline;

View File

@ -1051,17 +1051,27 @@ void StorageWindowView::threadFuncFireProc()
if (shutdown_called)
return;
/// Acquiring the lock can take seconds (depends on how long it takes to push) so we keep a reference to remember
/// what's the starting point where we want to push from
UInt32 timestamp_start = now();
std::lock_guard lock(fire_signal_mutex);
/// TODO: consider using time_t instead (for every timestamp in this class)
UInt32 timestamp_now = now();
LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}", timestamp_now, next_fire_signal, max_watermark);
LOG_TRACE(
log,
"Start: {}, now: {}, next fire signal: {}, max watermark: {}",
timestamp_start,
timestamp_now,
next_fire_signal,
max_watermark);
while (next_fire_signal <= timestamp_now)
{
try
{
if (max_watermark >= timestamp_now)
if (max_watermark >= timestamp_start)
fire(next_fire_signal);
}
catch (...)
@ -1075,11 +1085,18 @@ void StorageWindowView::threadFuncFireProc()
slide_interval *= 86400;
next_fire_signal += slide_interval;
LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}",
timestamp_now, next_fire_signal, max_watermark, max_fired_watermark, slide_interval);
LOG_TRACE(
log,
"Start: {}, now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}",
timestamp_start,
timestamp_now,
next_fire_signal,
max_watermark,
max_fired_watermark,
slide_interval);
}
if (max_watermark >= timestamp_now)
if (max_watermark >= timestamp_start)
clean_cache_task->schedule();
UInt64 next_fire_ms = static_cast<UInt64>(next_fire_signal) * 1000;

View File

@ -76,6 +76,21 @@ struct TableFunctionIcebergName
static constexpr auto name = "iceberg";
};
struct TableFunctionIcebergS3Name
{
static constexpr auto name = "icebergS3";
};
struct TableFunctionIcebergAzureName
{
static constexpr auto name = "icebergAzure";
};
struct TableFunctionIcebergLocalName
{
static constexpr auto name = "icebergLocal";
};
struct TableFunctionDeltaLakeName
{
static constexpr auto name = "deltaLake";
@ -86,14 +101,20 @@ struct TableFunctionHudiName
static constexpr auto name = "hudi";
};
#if USE_AWS_S3
#if USE_AVRO
# if USE_AWS_S3
using TableFunctionIceberg = ITableFunctionDataLake<TableFunctionIcebergName, StorageIceberg, TableFunctionS3>;
using TableFunctionIcebergS3 = ITableFunctionDataLake<TableFunctionIcebergS3Name, StorageIceberg, TableFunctionS3>;
# endif
# if USE_AZURE_BLOB_STORAGE
using TableFunctionIcebergAzure = ITableFunctionDataLake<TableFunctionIcebergAzureName, StorageIceberg, TableFunctionAzureBlob>;
# endif
using TableFunctionIcebergLocal = ITableFunctionDataLake<TableFunctionIcebergLocalName, StorageIceberg, TableFunctionLocal>;
#endif
#if USE_PARQUET
#if USE_AWS_S3
# if USE_PARQUET
using TableFunctionDeltaLake = ITableFunctionDataLake<TableFunctionDeltaLakeName, StorageDeltaLake, TableFunctionS3>;
#endif
using TableFunctionHudi = ITableFunctionDataLake<TableFunctionHudiName, StorageHudi, TableFunctionS3>;
#endif
}

View File

@ -14,10 +14,11 @@
#include <Storages/ObjectStorage/Utils.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <Storages/ObjectStorage/S3/Configuration.h>
#include <Storages/ObjectStorage/HDFS/Configuration.h>
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <Storages/ObjectStorage/Azure/Configuration.h>
#include <Storages/ObjectStorage/HDFS/Configuration.h>
#include <Storages/ObjectStorage/Local/Configuration.h>
#include <Storages/ObjectStorage/S3/Configuration.h>
#include <Storages/ObjectStorage/StorageObjectStorage.h>
namespace DB
@ -223,5 +224,5 @@ template class TableFunctionObjectStorage<OSSDefinition, StorageS3Configuration>
template class TableFunctionObjectStorage<HDFSDefinition, StorageHDFSConfiguration>;
template class TableFunctionObjectStorage<HDFSClusterDefinition, StorageHDFSConfiguration>;
#endif
template class TableFunctionObjectStorage<LocalDefinition, StorageLocalConfiguration>;
}

View File

@ -1,11 +1,11 @@
#pragma once
#include "config.h"
#include <TableFunctions/ITableFunction.h>
#include <Formats/FormatFactory.h>
#include <Disks/ObjectStorages/IObjectStorage_fwd.h>
#include <Storages/VirtualColumnUtils.h>
#include <Formats/FormatFactory.h>
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <Storages/VirtualColumnUtils.h>
#include <TableFunctions/ITableFunction.h>
#include "config.h"
namespace DB
{
@ -14,6 +14,7 @@ class Context;
class StorageS3Configuration;
class StorageAzureConfiguration;
class StorageHDFSConfiguration;
class StorageLocalConfiguration;
struct S3StorageSettings;
struct AzureStorageSettings;
struct HDFSStorageSettings;
@ -90,6 +91,17 @@ struct HDFSDefinition
static constexpr auto max_number_of_arguments = 4;
};
struct LocalDefinition
{
static constexpr auto name = "local";
static constexpr auto storage_type_name = "Local";
static constexpr auto signature = " - path\n"
" - path, format\n"
" - path, format, structure\n"
" - path, format, structure, compression_method\n";
static constexpr auto max_number_of_arguments = 4;
};
template <typename Definition, typename Configuration>
class TableFunctionObjectStorage : public ITableFunction
{
@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage<AzureDefinition, Stora
#if USE_HDFS
using TableFunctionHDFS = TableFunctionObjectStorage<HDFSDefinition, StorageHDFSConfiguration>;
#endif
using TableFunctionLocal = TableFunctionObjectStorage<LocalDefinition, StorageLocalConfiguration>;
}

View File

@ -4,24 +4,43 @@
namespace DB
{
#if USE_AWS_S3
#if USE_AVRO
void registerTableFunctionIceberg(TableFunctionFactory & factory)
{
# if USE_AWS_S3
factory.registerFunction<TableFunctionIceberg>(
{
.documentation =
{
.description=R"(The table function can be used to read the Iceberg table stored on object store.)",
{.documentation
= {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)",
.examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}},
.categories{"DataLake"}
},
.allow_readonly = false
});
.categories{"DataLake"}},
.allow_readonly = false});
factory.registerFunction<TableFunctionIcebergS3>(
{.documentation
= {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)",
.examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}},
.categories{"DataLake"}},
.allow_readonly = false});
# endif
# if USE_AZURE_BLOB_STORAGE
factory.registerFunction<TableFunctionIcebergAzure>(
{.documentation
= {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)",
.examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}},
.categories{"DataLake"}},
.allow_readonly = false});
# endif
factory.registerFunction<TableFunctionIcebergLocal>(
{.documentation
= {.description = R"(The table function can be used to read the Iceberg table stored locally.)",
.examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}},
.categories{"DataLake"}},
.allow_readonly = false});
}
#endif
#if USE_PARQUET
#if USE_AWS_S3
# if USE_PARQUET
void registerTableFunctionDeltaLake(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionDeltaLake>(
@ -55,11 +74,11 @@ void registerTableFunctionHudi(TableFunctionFactory & factory)
void registerDataLakeTableFunctions(TableFunctionFactory & factory)
{
UNUSED(factory);
#if USE_AWS_S3
#if USE_AVRO
registerTableFunctionIceberg(factory);
#endif
#if USE_PARQUET
#if USE_AWS_S3
# if USE_PARQUET
registerTableFunctionDeltaLake(factory);
#endif
registerTableFunctionHudi(factory);

View File

@ -286,4 +286,7 @@ class Utils:
@staticmethod
def is_job_triggered_manually():
return "robot" not in Envs.GITHUB_ACTOR
return (
"robot" not in Envs.GITHUB_ACTOR
and "clickhouse-ci" not in Envs.GITHUB_ACTOR
)

View File

@ -2,30 +2,92 @@ from minio import Minio
import glob
import os
import json
import shutil
def upload_directory(minio_client, bucket_name, local_path, s3_path):
result_files = []
for local_file in glob.glob(local_path + "/**"):
if os.path.isfile(local_file):
from enum import Enum
class CloudUploader:
def upload_directory(self, local_path, remote_blob_path, **kwargs):
print(kwargs)
result_files = []
# print(f"Arguments: {local_path}, {s3_path}")
# for local_file in glob.glob(local_path + "/**"):
# print("Local file: {}", local_file)
for local_file in glob.glob(local_path + "/**"):
result_local_path = os.path.join(local_path, local_file)
result_s3_path = os.path.join(s3_path, local_file)
print(f"Putting file {result_local_path} to {result_s3_path}")
minio_client.fput_object(
bucket_name=bucket_name,
object_name=result_s3_path,
file_path=result_local_path,
result_remote_blob_path = os.path.join(remote_blob_path, local_file)
if os.path.isfile(local_file):
self.upload_file(result_local_path, result_remote_blob_path, **kwargs)
result_files.append(result_remote_blob_path)
else:
files = self.upload_directory(
result_local_path, result_remote_blob_path, **kwargs
)
result_files.extend(files)
return result_files
class S3Uploader(CloudUploader):
def __init__(self, minio_client, bucket_name):
self.minio_client = minio_client
self.bucket_name = bucket_name
def upload_file(self, local_path, remote_blob_path, bucket=None):
print(f"Upload to bucket: {bucket}")
if bucket is None:
bucket = self.bucket_name
self.minio_client.fput_object(
bucket_name=bucket,
object_name=remote_blob_path,
file_path=local_path,
)
class LocalUploader(CloudUploader):
def __init__(self, clickhouse_node):
self.clickhouse_node = clickhouse_node
def upload_file(self, local_path, remote_blob_path):
dir_path = os.path.dirname(remote_blob_path)
if dir_path != "":
self.clickhouse_node.exec_in_container(
[
"bash",
"-c",
"mkdir -p {}".format(dir_path),
]
)
result_files.append(result_s3_path)
self.clickhouse_node.copy_file_to_container(local_path, remote_blob_path)
class AzureUploader(CloudUploader):
def __init__(self, blob_service_client, container_name):
self.blob_service_client = blob_service_client
self.container_client = self.blob_service_client.get_container_client(
container_name
)
def upload_file(self, local_path, remote_blob_path, container_name=None):
if container_name is None:
container_client = self.container_client
else:
files = upload_directory(
minio_client,
bucket_name,
os.path.join(local_path, local_file),
os.path.join(s3_path, local_file),
container_client = self.blob_service_client.get_container_client(
container_name
)
result_files.extend(files)
return result_files
blob_client = container_client.get_blob_client(remote_blob_path)
with open(local_path, "rb") as data:
blob_client.upload_blob(data, overwrite=True)
def upload_directory(minio_client, bucket, local_path, remote_path):
return S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory(
local_path, remote_path
)
def get_file_contents(minio_client, bucket, s3_path):

View File

@ -6,6 +6,7 @@ import logging
import os
import random
import shlex
import shutil
import signal
import string
import subprocess
@ -135,6 +136,53 @@ def check_args_and_update_paths(args):
)
def check_iptables_legacy():
iptables_path = shutil.which("iptables")
ip6tables_path = shutil.which("ip6tables")
if iptables_path is None:
print("Error: 'iptables' not found in PATH")
sys.exit(1)
if ip6tables_path is None:
print("Error: 'ip6tables' not found in PATH, ignoring")
try:
file_info = os.stat(iptables_path)
file_info_str = str(file_info)
if "legacy" in file_info_str:
print(
"""
iptables on your host machine is in 'legacy' mode. This is not supported.
Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual.
Or, use --ignore-iptables-legacy-check.
"""
)
sys.exit(1)
if not ip6tables_path:
return
file_info = os.stat(ip6tables_path)
file_info_str = str(file_info)
if "legacy" in file_info_str:
print(
"""
ip6tables on your host machine is in 'legacy' mode. This is not supported.
Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual.
Or, use --ignore-iptables-legacy-check.
"""
)
sys.exit(1)
except FileNotFoundError:
print(f"Error: '{iptables_path}' not found")
sys.exit(1)
def docker_kill_handler_handler(signum, frame):
_, _ = signum, frame
subprocess.check_call(
@ -163,6 +211,7 @@ if __name__ == "__main__":
level=logging.INFO,
format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)",
)
parser = argparse.ArgumentParser(description="ClickHouse integration tests runner")
parser.add_argument(
@ -311,12 +360,24 @@ if __name__ == "__main__":
help="Bind volume to this dir to use for dockerd files",
)
parser.add_argument(
"--ignore-iptables-legacy-check",
action="store_true",
default=False,
help="Ignore iptables-legacy usage check",
)
parser.add_argument("pytest_args", nargs="*", help="args for pytest command")
args = parser.parse_args()
check_args_and_update_paths(args)
if not args.ignore_iptables_legacy_check:
check_iptables_legacy()
else:
logging.warning("Skipping iptables-legacy check")
parallel_args = ""
if args.parallel:
parallel_args += "--dist=loadfile"

View File

@ -5,5 +5,11 @@
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3>
<azure>
<account_name>devstoreaccount1</account_name>
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
</azure>
<local>
</local>
</named_collections>
</clickhouse>

View File

@ -28,12 +28,15 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number
from pyspark.sql.window import Window
from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2
from minio.deleteobjects import DeleteObject
from azure.storage.blob import BlobServiceClient
from helpers.s3_tools import (
prepare_s3_bucket,
upload_directory,
get_file_contents,
list_s3_objects,
S3Uploader,
AzureUploader,
LocalUploader,
)
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@ -67,6 +70,7 @@ def started_cluster():
main_configs=["configs/config.d/named_collections.xml"],
user_configs=["configs/users.d/users.xml"],
with_minio=True,
with_azurite=True,
stay_alive=True,
)
@ -77,6 +81,25 @@ def started_cluster():
logging.info("S3 bucket created")
cluster.spark_session = get_spark()
cluster.default_s3_uploader = S3Uploader(
cluster.minio_client, cluster.minio_bucket
)
cluster.azure_container_name = "mycontainer"
cluster.blob_service_client = cluster.blob_service_client
container_client = cluster.blob_service_client.create_container(
cluster.azure_container_name
)
cluster.container_client = container_client
cluster.default_azure_uploader = AzureUploader(
cluster.blob_service_client, cluster.azure_container_name
)
cluster.default_local_uploader = LocalUploader(cluster.instances["node1"])
yield cluster
@ -142,12 +165,65 @@ def generate_data(spark, start, end):
return df
def create_iceberg_table(node, table_name, format="Parquet", bucket="root"):
def get_creation_expression(
storage_type,
table_name,
cluster,
format="Parquet",
table_function=False,
**kwargs,
):
if storage_type == "s3":
if "bucket" in kwargs:
bucket = kwargs["bucket"]
else:
bucket = cluster.minio_bucket
print(bucket)
if table_function:
return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')"
else:
return f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')"""
elif storage_type == "azure":
if table_function:
return f"""
icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})
"""
else:
return f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})"""
elif storage_type == "local":
if table_function:
return f"""
icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format})
"""
else:
return f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});"""
else:
raise Exception(f"Unknown iceberg storage type: {storage_type}")
def get_uuid_str():
return str(uuid.uuid4()).replace("-", "_")
def create_iceberg_table(
storage_type,
node,
table_name,
cluster,
format="Parquet",
**kwargs,
):
node.query(
f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')"""
get_creation_expression(storage_type, table_name, cluster, format, **kwargs)
)
@ -170,40 +246,69 @@ def create_initial_data_file(
return result_path
def default_upload_directory(
started_cluster, storage_type, local_path, remote_path, **kwargs
):
if storage_type == "local":
return started_cluster.default_local_uploader.upload_directory(
local_path, remote_path, **kwargs
)
elif storage_type == "s3":
print(kwargs)
return started_cluster.default_s3_uploader.upload_directory(
local_path, remote_path, **kwargs
)
elif storage_type == "azure":
return started_cluster.default_azure_uploader.upload_directory(
local_path, remote_path, **kwargs
)
else:
raise Exception(f"Unknown iceberg storage type: {storage_type}")
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_single_iceberg_file(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_single_iceberg_file(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_single_iceberg_file_" + format_version
inserted_data = "SELECT number, toString(number) as string FROM numbers(100)"
parquet_data_path = create_initial_data_file(
started_cluster, instance, inserted_data, TABLE_NAME
TABLE_NAME = (
"test_single_iceberg_file_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
write_iceberg_from_file(
spark, parquet_data_path, TABLE_NAME, format_version=format_version
write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME)
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
create_iceberg_table(instance, TABLE_NAME)
assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query(
inserted_data
"SELECT number, toString(number + 1) FROM numbers(100)"
)
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_partition_by(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_partition_by(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_partition_by_" + format_version
TABLE_NAME = (
"test_partition_by_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
write_iceberg_from_df(
spark,
@ -214,22 +319,33 @@ def test_partition_by(started_cluster, format_version):
partition_by="a",
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
assert len(files) == 14 # 10 partitiions + 4 metadata files
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_multiple_iceberg_files(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_multiple_iceberg_files(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_multiple_iceberg_files_" + format_version
TABLE_NAME = (
"test_multiple_iceberg_files_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
write_iceberg_from_df(
spark,
@ -239,9 +355,13 @@ def test_multiple_iceberg_files(started_cluster, format_version):
format_version=format_version,
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
# ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet',
# '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text',
# '/iceberg_data/default/test_multiple_iceberg_files/metadata/3127466b-299d-48ca-a367-6b9b1df1e78c-m0.avro',
@ -249,7 +369,7 @@ def test_multiple_iceberg_files(started_cluster, format_version):
# '/iceberg_data/default/test_multiple_iceberg_files/metadata/v1.metadata.json']
assert len(files) == 5
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
write_iceberg_from_df(
@ -259,8 +379,11 @@ def test_multiple_iceberg_files(started_cluster, format_version):
mode="append",
format_version=format_version,
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
assert len(files) == 9
@ -271,12 +394,13 @@ def test_multiple_iceberg_files(started_cluster, format_version):
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_types(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_types(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_types_" + format_version
TABLE_NAME = (
"test_types_" + format_version + "_" + storage_type + "_" + get_uuid_str()
)
data = [
(
@ -302,22 +426,29 @@ def test_types(started_cluster, format_version):
spark, df, TABLE_NAME, mode="overwrite", format_version=format_version
)
upload_directory(minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "")
default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1
assert (
instance.query(f"SELECT a, b, c, d, e FROM {TABLE_NAME}").strip()
== "123\tstring\t2000-01-01\t['str1','str2']\ttrue"
)
table_function = f"iceberg(s3, filename='iceberg_data/default/{TABLE_NAME}/')"
table_function_expr = get_creation_expression(
storage_type, TABLE_NAME, started_cluster, table_function=True
)
assert (
instance.query(f"SELECT a, b, c, d, e FROM {table_function}").strip()
instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip()
== "123\tstring\t2000-01-01\t['str1','str2']\ttrue"
)
assert instance.query(f"DESCRIBE {table_function} FORMAT TSV") == TSV(
assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV(
[
["a", "Nullable(Int32)"],
["b", "Nullable(String)"],
@ -329,12 +460,20 @@ def test_types(started_cluster, format_version):
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_delete_files(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_delete_files(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_delete_files_" + format_version
TABLE_NAME = (
"test_delete_files_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
write_iceberg_from_df(
spark,
@ -344,17 +483,22 @@ def test_delete_files(started_cluster, format_version):
format_version=format_version,
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0")
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0
@ -368,27 +512,41 @@ def test_delete_files(started_cluster, format_version):
format_version=format_version,
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150")
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_evolved_schema(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_evolved_schema(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_evolved_schema_" + format_version
TABLE_NAME = (
"test_evolved_schema_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
write_iceberg_from_df(
spark,
@ -398,19 +556,25 @@ def test_evolved_schema(started_cluster, format_version):
format_version=format_version,
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
expected_data = instance.query(f"SELECT * FROM {TABLE_NAME} order by a, b")
spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)")
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}")
@ -422,12 +586,13 @@ def test_evolved_schema(started_cluster, format_version):
assert data == expected_data
def test_row_based_deletes(started_cluster):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_row_based_deletes(started_cluster, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_row_based_deletes"
TABLE_NAME = "test_row_based_deletes_" + storage_type + "_" + get_uuid_str()
spark.sql(
f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')"
@ -436,17 +601,23 @@ def test_row_based_deletes(started_cluster):
f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)"
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10")
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
"",
)
error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}")
@ -454,13 +625,21 @@ def test_row_based_deletes(started_cluster):
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_schema_inference(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_schema_inference(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
for format in ["Parquet", "ORC", "Avro"]:
TABLE_NAME = "test_schema_inference_" + format + "_" + format_version
TABLE_NAME = (
"test_schema_inference_"
+ format
+ "_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
# Types time, timestamptz, fixed are not supported in Spark.
spark.sql(
@ -470,12 +649,16 @@ def test_schema_inference(started_cluster, format_version):
spark.sql(
f"insert into {TABLE_NAME} select 42, 4242, 42.42, 4242.4242, decimal(42.42), decimal(42.42), decimal(42.42), date('2020-01-01'), timestamp('2020-01-01 20:00:00'), 'hello', binary('hello'), array(1,2,3), map('key', 'value'), struct(42, 'hello'), array(struct(map('key', array(map('key', 42))), struct(42, 'hello')))"
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME, format)
create_iceberg_table(
storage_type, instance, TABLE_NAME, started_cluster, format=format
)
res = instance.query(
f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0}
@ -510,12 +693,18 @@ def test_schema_inference(started_cluster, format_version):
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_metadata_file_selection(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_metadata_file_selection(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_metadata_selection_" + format_version
TABLE_NAME = (
"test_metadata_selection_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
spark.sql(
f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')"
@ -526,22 +715,31 @@ def test_metadata_file_selection(started_cluster, format_version):
f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)"
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500
@pytest.mark.parametrize("format_version", ["1", "2"])
def test_metadata_file_format_with_uuid(started_cluster, format_version):
@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"])
def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
minio_client = started_cluster.minio_client
bucket = started_cluster.minio_bucket
TABLE_NAME = "test_metadata_selection_with_uuid_" + format_version
TABLE_NAME = (
"test_metadata_selection_with_uuid_"
+ format_version
+ "_"
+ storage_type
+ "_"
+ get_uuid_str()
)
spark.sql(
f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')"
@ -555,40 +753,48 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version):
for i in range(50):
os.rename(
f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json",
f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json",
f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{get_uuid_str()}.metadata.json",
)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)
create_iceberg_table(instance, TABLE_NAME)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500
def test_restart_broken(started_cluster):
def test_restart_broken_s3(started_cluster):
instance = started_cluster.instances["node1"]
spark = started_cluster.spark_session
TABLE_NAME = "test_restart_broken_table_function_s3" + "_" + get_uuid_str()
minio_client = started_cluster.minio_client
bucket = "broken2"
TABLE_NAME = "test_restart_broken_table_function"
if not minio_client.bucket_exists(bucket):
minio_client.make_bucket(bucket)
parquet_data_path = create_initial_data_file(
started_cluster,
instance,
"SELECT number, toString(number) FROM numbers(100)",
write_iceberg_from_df(
spark,
generate_data(spark, 0, 100),
TABLE_NAME,
mode="overwrite",
format_version="1",
)
write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1")
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
"s3",
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
bucket=bucket,
)
create_iceberg_table(instance, TABLE_NAME, bucket=bucket)
create_iceberg_table("s3", instance, TABLE_NAME, started_cluster, bucket=bucket)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100
s3_objects = list_s3_objects(minio_client, bucket, prefix="")
@ -613,8 +819,12 @@ def test_restart_broken(started_cluster):
minio_client.make_bucket(bucket)
files = upload_directory(
minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", ""
files = default_upload_directory(
started_cluster,
"s3",
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
bucket=bucket,
)
assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100

View File

@ -8,58 +8,64 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS contributors;
CREATE TABLE contributors (name String) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT Parquet"
# random results
${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null
${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors"
${CLICKHOUSE_CLIENT} -n --query="
-- random results
SELECT * FROM contributors LIMIT 10 FORMAT Null;
DROP TABLE contributors;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_numbers"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_numbers;
CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory;"
# less than default block size (65k)
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
TRUNCATE TABLE parquet_numbers;"
# More than default block size
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
TRUNCATE TABLE parquet_numbers;"
${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
TRUNCATE TABLE parquet_numbers;"
${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
DROP TABLE parquet_numbers;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_events"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_events;
CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_events FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_events"
${CLICKHOUSE_CLIENT} -n --query="
SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event;
DROP TABLE parquet_events;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types4"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
# convert min type
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory"
# convert max type
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_types1;
DROP TABLE IF EXISTS parquet_types2;
DROP TABLE IF EXISTS parquet_types3;
DROP TABLE IF EXISTS parquet_types4;
CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
-- convert min type
CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory;
-- convert max type
CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9))"
INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9));
# min
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9))"
-- min
INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9));
# max
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9))"
-- max
INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9));"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
@ -72,8 +78,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8 FORMAT
echo diff:
diff "${CLICKHOUSE_TMP}"/parquet_all_types_1.dump "${CLICKHOUSE_TMP}"/parquet_all_types_2.dump
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9))"
${CLICKHOUSE_CLIENT} -n --query="
TRUNCATE TABLE parquet_types2;
INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9));"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 FORMAT Parquet"
@ -81,70 +88,69 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80,
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 FORMAT Parquet"
echo dest:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8"
echo min:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8"
echo max:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8"
${CLICKHOUSE_CLIENT} -n --query="
SELECT 'dest:';
SELECT * FROM parquet_types2 ORDER BY int8;
SELECT 'min:';
SELECT * FROM parquet_types3 ORDER BY int8;
SELECT 'max:';
SELECT * FROM parquet_types4 ORDER BY int8;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types5"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types6"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
DROP TABLE IF EXISTS parquet_types5;
DROP TABLE IF EXISTS parquet_types6;
TRUNCATE TABLE parquet_types2;
CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory;
CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory;
INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
echo dest from null:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types6 ORDER BY int8"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_types6 ORDER BY int8;
DROP TABLE parquet_types5;
DROP TABLE parquet_types6;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types5"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types6"
DROP TABLE parquet_types1;
DROP TABLE parquet_types2;
DROP TABLE parquet_types3;
DROP TABLE parquet_types4;
DROP TABLE IF EXISTS parquet_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types3"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4"
CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory();
INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001]);
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [])"
INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], []);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays ORDER BY id"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_arrays ORDER BY id;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_arrays"
DROP TABLE parquet_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nullable_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], [])"
DROP TABLE IF EXISTS parquet_nullable_arrays;
CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory();
INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], []);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays ORDER BY id"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nullable_arrays"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_nullable_arrays ORDER BY id;
DROP TABLE parquet_nullable_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nested_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory() "
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []])"
DROP TABLE IF EXISTS parquet_nested_arrays;
CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory();
INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []]);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nested_arrays"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_nested_arrays;
DROP TABLE parquet_nested_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_decimal"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123)"
DROP TABLE IF EXISTS parquet_decimal;
CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory;
INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_decimal FORMAT Arrow"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_decimal"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_decimal;
DROP TABLE parquet_decimal;"

View File

@ -6,108 +6,115 @@
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
# Simple small values
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
-- Simple small values
CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);
INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);
INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal0.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump
echo diff0:
diff "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (0.000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-0.000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');"
CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);
INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);
INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);
INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);
INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);
INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);
INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);
INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);
INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);
INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);
INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);
INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);
INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);
INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);
INSERT INTO decimal (i) VALUES (0.000000000000000001);
INSERT INTO decimal (i) VALUES (-0.000000000000000001);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);
INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal1.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump
echo diff1:
diff "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);"
CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal2.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump
echo diff2:
diff "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
# Empty table test
# throws No data to insert
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" 2> /dev/null
echo nothing:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM decimal2 ORDER BY a, b, c, d;
TRUNCATE TABLE decimal2;
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, Null, Null)"
INSERT INTO decimal VALUES (Null, Null, Null, Null);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
echo nulls:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM decimal2 ORDER BY a, b, c, d;
TRUNCATE TABLE decimal2;
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (1, Null, Null, Null)"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, 1, Null, Null)"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, 1, Null)"
INSERT INTO decimal VALUES (1, Null, Null, Null);
INSERT INTO decimal VALUES (Null, 1, Null, Null);
INSERT INTO decimal VALUES (Null, Null, 1, Null);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_3.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
echo full orig:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d;"
echo full inserted:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT 'full orig:';
SELECT * FROM decimal ORDER BY a, b, c, d;
SELECT 'full inserted:';
SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump
echo diff3:
diff "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;"

View File

@ -57,14 +57,14 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A
# COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.py $JSON` 2>&1 || continue
COLUMNS=$(cat "$COLUMNS_FILE") || continue
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load"
$CLICKHOUSE_CLIENT <<EOF
CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory;
EOF
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS parquet_load;
CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory;"
# Some files contain unsupported data structures, exception is ok.
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" < "$DATA_DIR"/"$NAME" 2>&1 | sed 's/Exception/Ex---tion/'
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100;
DROP TABLE parquet_load;"
done

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -42,6 +41,7 @@ function check()
while [ "$query_result" != "2.2" ]
do
sleep 0.2
query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))")
done
}

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -20,7 +20,9 @@ with client(name="client1>", log=log) as client1, client(
client1.expect(prompt)
client2.expect(prompt)
client1.send("SET enable_analyzer = 0")
client1.send(
"SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000"
)
client1.expect(prompt)
client1.send("SET allow_experimental_window_view = 1")
client1.expect(prompt)
@ -28,7 +30,9 @@ with client(name="client1>", log=log) as client1, client(
client1.expect(prompt)
client2.send("SET allow_experimental_window_view = 1")
client2.expect(prompt)
client2.send("SET enable_analyzer = 0")
client2.send(
"SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000"
)
client2.expect(prompt)
client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch")
@ -42,8 +46,9 @@ with client(name="client1>", log=log) as client1, client(
"CREATE TABLE 01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
)
client1.expect(prompt)
# Introduce a sleep call to verify that even if the push to view is slow WATCH will work
client1.send(
"CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
"CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(sleep(5)) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
)
client1.expect(prompt)

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
-- Tags: no-parallel
-- no-parallel: Slows down query_log
DROP TABLE IF EXISTS slow_log;
DROP TABLE IF EXISTS expected_times;

View File

@ -4,6 +4,8 @@ Invocation with constant
2020-11-01
\N
\N
0000-01-01
9999-12-31
or null
2020-11-01
\N

View File

@ -7,6 +7,8 @@ SELECT fromModifiedJulianDay(59154);
SELECT fromModifiedJulianDay(NULL);
SELECT fromModifiedJulianDay(CAST(NULL, 'Nullable(Int64)'));
SELECT fromModifiedJulianDay(-678942); -- { serverError CANNOT_FORMAT_DATETIME }
SELECT fromModifiedJulianDay(-678941);
SELECT fromModifiedJulianDay(2973483);
SELECT fromModifiedJulianDay(2973484); -- { serverError CANNOT_FORMAT_DATETIME }
SELECT 'or null';

View File

@ -1,5 +1,6 @@
#!/usr/bin/env bash
# Tags: no-parallel
# Tags: no-fasttest
# no-fasttest: It's a bit demanding
# Creation of a database with Ordinary engine emits a warning.
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
@ -9,9 +10,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
## tests with real clickhouse restart would be a bit to heavy,
## to ensure the table will not reappear back clickhose-local is enough.
## to ensure the table will not reappear back clickhouse-local is enough.
WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/01600_detach_permanently"
WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/${CLICKHOUSE_TEST_UNIQUE_NAME}"
rm -rf "${WORKING_FOLDER_01600}"
mkdir -p "${WORKING_FOLDER_01600}"

View File

@ -11,13 +11,21 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDE
$CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)"
for _ in {0..30}; do
run_query() {
output=`$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_index_hypothesis WHERE a = b"`
if [[ $output != "0" ]]; then
echo "output: $output, expected: 0"
exit 1
fi
done
}
export -f run_query
parallel -j 8 run_query ::: {0..30}
if [ $? -ne 0 ]; then
echo FAILED
exit 1
fi
echo OK

View File

@ -1,48 +1,51 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
ROLE="r_${CLICKHOUSE_TEST_UNIQUE_NAME}"
USER="u_${CLICKHOUSE_TEST_UNIQUE_NAME}"
QUOTA="q_${CLICKHOUSE_TEST_UNIQUE_NAME}"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247"
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247"
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247"
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}"
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE written_bytes_02247(s String) ENGINE = Memory"
${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02247"
${CLICKHOUSE_CLIENT} -q "CREATE USER u02247"
${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247"
${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO r02247"
${CLICKHOUSE_CLIENT} -q "CREATE ROLE ${ROLE}"
${CLICKHOUSE_CLIENT} -q "CREATE USER ${USER}"
${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO ${ROLE}"
${CLICKHOUSE_CLIENT} -q "GRANT ${ROLE} to ${USER}"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO ${ROLE}"
# The value 'qwqw' means about 13 bytes are to be written, so the current quota (30 bytes) gives the ability to write 'qwqw' 2 times.
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = 'q02247'"
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'"
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA q02247"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA ${QUOTA}"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO ${ROLE}"
${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247"
# Numbers from 0 to 50 means about 540 bytes are to be written, so the current quota (1000 bytes) is enough to do so.
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)"
${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)"
# Numbers from 0 to 100 means about 1090 bytes are to be written, so the current quota (1000 bytes total - 540 bytes already used) is NOT enough to do so.
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'"
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'"
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247"
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247"
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247"
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}"
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}"
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}"

View File

@ -1,11 +1,10 @@
#!/usr/bin/env bash
# Tags: no-parallel
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz
ln -s "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz
${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=1000000000"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=100000000"

View File

@ -1,9 +1,11 @@
-- Tags: long, no-parallel
DROP TABLE IF EXISTS t;
create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18
SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8092, index_granularity_bytes='10Mi';
SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8192, index_granularity_bytes='10Mi';
insert into t (c1, c18) select number, -number from numbers(2000000);
insert into t (c1, c18) select number, -number from numbers(500000);
alter table t add projection p_norm (select * order by c1);

View File

@ -1,6 +1,6 @@
Test 1: create filesystem database and check implicit calls
0
test1
default_test1
4
4
30

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -15,14 +14,11 @@ echo '2,"def",456,"bacabaa"' >> ${user_files_tmp_dir}/tmp.csv
echo '3,"story",78912,"acabaab"' >> ${user_files_tmp_dir}/tmp.csv
echo '4,"history",21321321,"cabaaba"' >> ${user_files_tmp_dir}/tmp.csv
tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME}
$CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_1.csv') select * from numbers(1, 10)"
$CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_2.csv') select * from numbers(11, 10)"
$CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_30.csv') select * from numbers(21, 10)"
[[ -d $tmp_dir ]] && rm -rd $tmp_dir
mkdir $tmp_dir
tmp_dir=$(mktemp -d ${CLICKHOUSE_TEST_UNIQUE_NAME}_XXXX)
cp ${user_files_tmp_dir}/tmp.csv ${tmp_dir}/tmp.csv
cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp/tmp.csv
cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp.myext
@ -30,55 +26,57 @@ cp ${user_files_tmp_dir}/tmp_numbers_1.csv ${user_files_tmp_dir}/tmp/tmp_numbers
#################
echo "Test 1: create filesystem database and check implicit calls"
DATABASE_TEST1="${CLICKHOUSE_DATABASE}_test1"
${CLICKHOUSE_CLIENT} --multiline -q """
DROP DATABASE IF EXISTS test1;
CREATE DATABASE test1 ENGINE = Filesystem;
DROP DATABASE IF EXISTS ${DATABASE_TEST1};
CREATE DATABASE ${DATABASE_TEST1} ENGINE = Filesystem;
"""
echo $?
${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/tmp.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/*/*tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "${DATABASE_TEST1}"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/tmp.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/*/*tmp_numbers_*.csv\`;"
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')";
# **/* does not search in the current directory but searches recursively in nested directories.
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')";
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\""
#################
echo "Test 2: check DatabaseFilesystem access rights and errors handling on server"
# DATABASE_ACCESS_DENIED: Allows list files only inside user_files
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --multiline --query """
USE test1;
USE ${DATABASE_TEST1};
SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\";
""" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||:
# BAD_ARGUMENTS: path should be inside user_files
DATABASE_TEST2="${CLICKHOUSE_DATABASE}_test2"
${CLICKHOUSE_CLIENT} --multiline -q """
DROP DATABASE IF EXISTS test2;
CREATE DATABASE test2 ENGINE = Filesystem('/tmp');
DROP DATABASE IF EXISTS ${DATABASE_TEST2};
CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('/tmp');
""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||:
# BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist
${CLICKHOUSE_CLIENT} --multiline -q """
DROP DATABASE IF EXISTS test2;
CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir');
DROP DATABASE IF EXISTS ${DATABASE_TEST2};
CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('relative_unknown_dir');
""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||:
# FILE_DOESNT_EXIST: unknown file
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||:
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||:
# Clean
${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS ${DATABASE_TEST1};"
rm -rd $tmp_dir
rm -rd $user_files_tmp_dir

View File

@ -1,13 +1,12 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
QUOTA="2884_quota_$$"
USER="2884_user_$$"
ROLE="2884_role_$$"
QUOTA="2884_quota_${CLICKHOUSE_DATABASE}"
USER="2884_user_${CLICKHOUSE_DATABASE}"
ROLE="2884_role_${CLICKHOUSE_DATABASE}"
function login_test()
@ -26,10 +25,10 @@ function login_test()
echo "> Alter the quota with MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4"
${CLICKHOUSE_CLIENT} -q "ALTER QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4 TO ${USER}"
echo "> Try to login to the user account with correct password"
${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null"
echo "> Successfull login should reset failed authentications counter. Check the failed_sequential_authentications, max_failed_sequential_authentications fields."
${CLICKHOUSE_CLIENT} -q "SELECT failed_sequential_authentications, max_failed_sequential_authentications FROM system.quotas_usage WHERE quota_name = '${QUOTA}'"
@ -39,7 +38,7 @@ function login_test()
${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect'
${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect'
${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED'
echo "> Also try to login with correct password. Quota should stay exceeded."
${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED'

View File

@ -1,51 +0,0 @@
CREATE TABLE t_02967
(
`key` Date,
`value` UInt16
)
ENGINE = MergeTree
ORDER BY key
SETTINGS
index_granularity_bytes = 0 --8192 --, min_index_granularity_bytes = 2
, index_granularity = 100
, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0
--
-- , min_bytes_for_wide_part = 2
AS SELECT
number,
repeat(toString(number), 5)
FROM numbers(105.);
-- Check with newly inserted data part. It's in-memory structured are filled at insert time.
SELECT
count(ignore(*))
FROM t_02967
PREWHERE CAST(ignore() + 1 as UInt8)
GROUP BY
ignore(65535, *),
ignore(255, 256, *)
SETTINGS
--send_logs_level='test',
max_threads=1;
-- Reload part form disk to check that in-meory structures where properly serilaized-deserialized
DETACH TABLE t_02967;
ATTACH TABLE t_02967;
SELECT
count(ignore(*))
FROM t_02967
PREWHERE CAST(ignore() + 1 as UInt8)
GROUP BY
ignore(65535, *),
ignore(255, 256, *)
SETTINGS
--send_logs_level='test',
max_threads=1;
DROP TABLE t_02967;

View File

@ -1,6 +1,7 @@
#!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel, no-shared-merge-tree
# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest
# SMT: The merge process is completely different from RMT
# no-fasttest: Avoid long waits
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -12,8 +13,7 @@ function wait_until()
{
local q=$1 && shift
while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do
# too frequent FLUSH LOGS is too costly
sleep 2
sleep 0.5
done
}
@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT -m -q "
system start replicated sends rmt_master;
"
# wait until rmt_slave will fetch the part and reflect this error in system.part_log
wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0"
wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0"
$CLICKHOUSE_CLIENT -m -q "
system sync replica rmt_slave;

View File

@ -1,6 +1,7 @@
#!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel, no-shared-merge-tree
# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest
# SMT: The merge process is completely different from RMT
# no-fasttest: Avoid long waits
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -15,8 +16,7 @@ function wait_until()
{
local q=$1 && shift
while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do
# too frequent FLUSH LOGS is too costly
sleep 2
sleep 0.5
done
}
@ -43,9 +43,8 @@ $CLICKHOUSE_CLIENT -m -q "
wait_for_mutation rmt_master 0000000000
$CLICKHOUSE_CLIENT -m -q "system start pulling replication log rmt_slave"
# and wait until rmt_slave to fetch the part and reflect this error in system.part_log
wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0"
wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0"
$CLICKHOUSE_CLIENT -m -q "
system flush logs;
select 'before';
select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3;

View File

@ -1,12 +0,0 @@
DROP TABLE IF EXISTS data_02051__fuzz_24;
CREATE TABLE data_02051__fuzz_24 (`key` Int16, `value` String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part=0 AS SELECT number, repeat(toString(number), 5) FROM numbers(1000000.);
SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*);
detach table data_02051__fuzz_24;
attach table data_02051__fuzz_24;
SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*);
DROP TABLE data_02051__fuzz_24;

View File

@ -0,0 +1,2 @@
50000
1

View File

@ -0,0 +1,20 @@
CREATE TABLE IF NOT EXISTS table_name
(
id UInt64
)
ENGINE = MergeTree()
ORDER BY cityHash64(id)
SAMPLE BY cityHash64(id);
INSERT INTO table_name SELECT rand() from system.numbers limit 10000;
INSERT INTO table_name SELECT rand() from system.numbers limit 10000;
INSERT INTO table_name SELECT rand() from system.numbers limit 10000;
INSERT INTO table_name SELECT rand() from system.numbers limit 10000;
INSERT INTO table_name SELECT rand() from system.numbers limit 10000;
select count() from table_name;
SELECT count() < 50 * 5 FROM (
SELECT * FROM table_name SAMPLE 50
);
DROP TABLE table_name;

View File

@ -0,0 +1,13 @@
11
12
13
14
15
16
17
18
19
20
21
22
23

View File

@ -0,0 +1,36 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS t_async_insert_params;
CREATE TABLE t_async_insert_params (id UInt64) ENGINE = Memory;
"
cmd_params="--async_insert 1 --async_insert_busy_timeout_max_ms 300000 --async_insert_busy_timeout_min_ms 300000 --wait_for_async_insert 0 --async_insert_use_adaptive_busy_timeout 0"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p1 = 11; INSERT INTO t_async_insert_params VALUES ({p1:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 12; INSERT INTO t_async_insert_params VALUES ({p2:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 1000; INSERT INTO t_async_insert_params VALUES (13);"
$CLICKHOUSE_CLIENT $cmd_params -q 'SET param_p2 = 1000; INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 14};'
$CLICKHOUSE_CLIENT $cmd_params --param_p1 15 -q "INSERT INTO t_async_insert_params VALUES ({p1:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 16 -q "INSERT INTO t_async_insert_params VALUES ({p2:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q "INSERT INTO t_async_insert_params VALUES (17);"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 18};'
url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0"
${CLICKHOUSE_CURL} -sS "$url&param_p1=19" -d "INSERT INTO t_async_insert_params VALUES ({p1:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p2=20" -d "INSERT INTO t_async_insert_params VALUES ({p2:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p3=21" -d "INSERT INTO t_async_insert_params VALUES ({p3:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p2=1000" -d "INSERT INTO t_async_insert_params VALUES (22)"
${CLICKHOUSE_CURL} -sS "$url&param_p2=1000" -d 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 23}'
$CLICKHOUSE_CLIENT -q "
SYSTEM FLUSH ASYNC INSERT QUEUE;
SELECT id FROM t_async_insert_params ORDER BY id;
DROP TABLE IF EXISTS t_async_insert_params;
"

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS t_async_insert_params;
CREATE TABLE t_async_insert_params (id UInt64) ENGINE = MergeTree ORDER BY tuple();
SET param_p1 = 'Hello';
SET async_insert = 1;
SET wait_for_async_insert = 1;
INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER }
INSERT INTO t_async_insert_params VALUES ({p1:String}); -- { serverError TYPE_MISMATCH }
ALTER TABLE t_async_insert_params MODIFY COLUMN id String;
INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER }
INSERT INTO t_async_insert_params VALUES ({p1:String});
SELECT * FROM t_async_insert_params ORDER BY id;
DROP TABLE t_async_insert_params;

View File

@ -0,0 +1,2 @@
Map(LowCardinality(String), LowCardinality(String))
1 1

View File

@ -0,0 +1,7 @@
SELECT toTypeName(_headers)
FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String');
SELECT
*,
mapFromString(_headers['X-ClickHouse-Summary'])['read_rows']
FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String');

View File

@ -0,0 +1,2 @@
{"a":"42"} JSON
{"a":42} Object(\'json\')

View File

@ -0,0 +1,8 @@
set allow_experimental_object_type=1;
set allow_experimental_json_type=1;
set use_json_alias_for_old_object_type=0;
select materialize('{"a" : 42}')::JSON as json, toTypeName(json);
set use_json_alias_for_old_object_type=1;
select '{"a" : 42}'::JSON as json, toTypeName(json);
select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS}

View File

@ -1,5 +1,5 @@
test hits 1 57344 7
test hits 1 8832938 1079
test hits 1 829354 102
test hits 1 8839168 1079
test hits 1 835584 102
test hits 1 8003584 977
test hits 2 581632 71