Merge remote-tracking branch 'blessed/master' into libcxx13_take2

This commit is contained in:
Raúl Marín 2021-12-14 13:05:01 +01:00
commit 44f3b1c9d2
115 changed files with 3341 additions and 829 deletions

3
.gitmodules vendored
View File

@ -247,3 +247,6 @@
[submodule "contrib/sysroot"]
path = contrib/sysroot
url = https://github.com/ClickHouse-Extras/sysroot.git
[submodule "contrib/azure"]
path = contrib/azure
url = https://github.com/ClickHouse-Extras/azure-sdk-for-cpp.git

View File

@ -1,4 +1,4 @@
### ClickHouse release v21.12, 2021-12-13
### ClickHouse release v21.12, 2021-12-15
#### Backward Incompatible Change

View File

@ -447,7 +447,7 @@ if (MAKE_STATIC_LIBRARIES)
# It's disabled for ARM because otherwise ClickHouse cannot run on Android.
set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie")
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no-pie")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie")
endif ()
else ()
set (CMAKE_POSITION_INDEPENDENT_CODE ON)
@ -508,6 +508,7 @@ include (cmake/find/hdfs3.cmake) # uses protobuf
include (cmake/find/poco.cmake)
include (cmake/find/curl.cmake)
include (cmake/find/s3.cmake)
include (cmake/find/blob_storage.cmake)
include (cmake/find/base64.cmake)
include (cmake/find/parquet.cmake)
include (cmake/find/simdjson.cmake)

View File

@ -2,7 +2,13 @@
ClickHouse is an open project, and you can contribute to it in many ways. You can help with ideas, code, or documentation. We appreciate any efforts that help us to make the project better.
Thank you.
Thank you!
## Legal Info
When you open your first pull-request to ClickHouse repo, a bot will invite you to accept ClickHouse Individual CLA (Contributor License Agreement). It is a simple few click process. For subsequent pull-requests the bot will check if you have already signed it and won't bother you again.
Optionally, to make contributions even more tight legally, your employer as a legal entity may want to sign a ClickHouse Corporate CLA with ClickHouse, Inc. If you're interested to do so, contact us at [legal@clickhouse.com](mailto:legal@clickhouse.com).
## Technical Info

View File

@ -0,0 +1,28 @@
option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY
"Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)"
ON)
if (USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY)
set(USE_AZURE_BLOB_STORAGE 1)
set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk)
endif()
if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk"
OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules")
AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY)
message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init")
set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF)
set(USE_AZURE_BLOB_STORAGE 0)
endif ()
if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY)
message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library")
endif()
if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY)
message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library")
endif()
if (USE_AZURE_BLOB_STORAGE)
message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}")
endif()

View File

@ -249,6 +249,10 @@ endif()
# - sentry-native
add_subdirectory (curl-cmake)
if (USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY)
add_subdirectory(azure-cmake)
endif()
if (USE_SENTRY)
add_subdirectory (sentry-native-cmake)
endif()

1
contrib/azure vendored Submodule

@ -0,0 +1 @@
Subproject commit ac4b763d4ca40122275f1497cbdc5451337461d9

View File

@ -0,0 +1,71 @@
set(AZURE_DIR "${ClickHouse_SOURCE_DIR}/contrib/azure")
set(AZURE_SDK_LIBRARY_DIR "${AZURE_DIR}/sdk")
file(GLOB AZURE_SDK_CORE_SRC
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/cryptography/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/http/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/http/curl/*.hpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/http/curl/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/winhttp/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/io/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/src/private/*.hpp"
)
file(GLOB AZURE_SDK_IDENTITY_SRC
"${AZURE_SDK_LIBRARY_DIR}/identity/azure-identity/src/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/identity/azure-identity/src/private/*.hpp"
)
file(GLOB AZURE_SDK_STORAGE_COMMON_SRC
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-common/src/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-common/src/private/*.cpp"
)
file(GLOB AZURE_SDK_STORAGE_BLOBS_SRC
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-blobs/src/*.cpp"
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-blobs/src/private/*.hpp"
)
file(GLOB AZURE_SDK_UNIFIED_SRC
${AZURE_SDK_CORE_SRC}
${AZURE_SDK_IDENTITY_SRC}
${AZURE_SDK_STORAGE_COMMON_SRC}
${AZURE_SDK_STORAGE_BLOBS_SRC}
)
set(AZURE_SDK_INCLUDES
"${AZURE_SDK_LIBRARY_DIR}/core/azure-core/inc/"
"${AZURE_SDK_LIBRARY_DIR}/identity/azure-identity/inc/"
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-common/inc/"
"${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-blobs/inc/"
)
include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake")
add_library(azure_sdk ${AZURE_SDK_UNIFIED_SRC})
if (COMPILER_CLANG)
target_compile_options(azure_sdk PUBLIC
-Wno-deprecated-copy-dtor
-Wno-extra-semi
-Wno-suggest-destructor-override
-Wno-inconsistent-missing-destructor-override
-Wno-error=unknown-warning-option
-Wno-reserved-identifier
)
endif()
# Originally, on Windows azure-core is built with bcrypt and crypt32 by default
if (OPENSSL_FOUND)
target_link_libraries(azure_sdk PRIVATE ${OPENSSL_LIBRARIES})
endif()
# Originally, on Windows azure-core is built with winhttp by default
if (CURL_FOUND)
target_link_libraries(azure_sdk PRIVATE ${CURL_LIBRARY})
endif()
target_link_libraries(azure_sdk PRIVATE ${LIBXML2_LIBRARIES})
target_include_directories(azure_sdk PUBLIC ${AZURE_SDK_INCLUDES})

View File

@ -639,6 +639,7 @@ add_library(
"${BORINGSSL_SOURCE_DIR}/decrepit/ssl/ssl_decrepit.c"
"${BORINGSSL_SOURCE_DIR}/decrepit/cfb/cfb.c"
"${BORINGSSL_SOURCE_DIR}/decrepit/bio/base64_bio.c"
)
add_executable(

View File

@ -85,7 +85,8 @@ RUN python3 -m pip install \
tzlocal==2.1 \
urllib3 \
requests-kerberos \
pyhdfs
pyhdfs \
azure-storage-blob
COPY modprobe.sh /usr/local/bin/modprobe
COPY dockerd-entrypoint.sh /usr/local/bin/

View File

@ -0,0 +1,13 @@
version: '2.3'
services:
azurite1:
image: mcr.microsoft.com/azure-storage/azurite
ports:
- "10000:10000"
volumes:
- data1-1:/data1
command: azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log
volumes:
data1-1:

View File

@ -17,6 +17,7 @@ ClickHouse server works as MySQL replica. It reads binlog and performs DDL and D
``` sql
CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster]
ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...]
[TABLE OVERRIDE table1 (...), TABLE OVERRIDE table2 (...)]
```
**Engine Parameters**
@ -109,15 +110,19 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([
- MySQL `DELETE` query is converted into `INSERT` with `_sign=-1`.
- MySQL `UPDATE` query is converted into `INSERT` with `_sign=-1` and `INSERT` with `_sign=1`.
- MySQL `UPDATE` query is converted into `INSERT` with `_sign=-1` and `INSERT` with `_sign=1` if the primary key has been changed, or
`INSERT` with `_sign=1` if not.
### Selecting from MaterializedMySQL Tables {#select}
`SELECT` query from `MaterializedMySQL` tables has some specifics:
- If `_version` is not specified in the `SELECT` query, [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used. So only rows with `MAX(_version)` are selected.
- If `_version` is not specified in the `SELECT` query, the
[FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used, so only rows with
`MAX(_version)` are returned for each primary key value.
- If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not included into the result set.
- If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not
included into the result set.
- The result includes columns comments in case they exist in MySQL database tables.
@ -125,15 +130,77 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([
MySQL `PRIMARY KEY` and `INDEX` clauses are converted into `ORDER BY` tuples in ClickHouse tables.
ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use [materialized views](../../sql-reference/statements/create/view.md#materialized).
ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use
[materialized views](../../sql-reference/statements/create/view.md#materialized).
**Notes**
- Rows with `_sign=-1` are not deleted physically from the tables.
- Cascade `UPDATE/DELETE` queries are not supported by the `MaterializedMySQL` engine.
- Cascade `UPDATE/DELETE` queries are not supported by the `MaterializedMySQL` engine, as they are not visible in the
MySQL binlog.
- Replication can be easily broken.
- Manual operations on database and tables are forbidden.
- `MaterializedMySQL` is influenced by [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert) setting. The data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL server changes.
- `MaterializedMySQL` is affected by the [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert)
setting. Data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL
server changes.
### Table Overrides {#table-overrides}
Table overrides can be used to customize the ClickHouse DDL queries, allowing you to make schema optimizations for your
application. This is especially useful for controlling partitioning, which is important for the overall performance of
MaterializedMySQL.
```sql
CREATE DATABASE db_name ENGINE = MaterializedMySQL(...)
[SETTINGS ...]
[TABLE OVERRIDE table_name (
[COLUMNS (
[name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], ...]
[INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, ...]
[PROJECTION projection_name_1 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]), ...]
)]
[ORDER BY expr]
[PRIMARY KEY expr]
[PARTITION BY expr]
[SAMPLE BY expr]
[TTL expr]
), ...]
```
Example:
```sql
CREATE DATABASE db_name ENGINE = MaterializedMySQL(...)
TABLE OVERRIDE table1 (
COLUMNS (
userid UUID,
category LowCardinality(String),
timestamp DateTime CODEC(Delta, Default)
)
PARTITION BY toYear(timestamp)
),
TABLE OVERRIDE table2 (
COLUMNS (
ip_hash UInt32 MATERIALIZED xxHash32(client_ip),
client_ip String TTL created + INTERVAL 72 HOUR
)
SAMPLE BY ip_hash
)
```
The `COLUMNS` list is sparse; it contains only modified or extra (MATERIALIZED or ALIAS) columns. Modified columns with
a different type must be assignable from the original type. There is currently no validation of this or similar issues
when the `CREATE DATABASE` query executes, so extra care needs to be taken.
You may specify overrides for tables that do not exist yet.
!!! note "Warning"
It is easy to break replication with TABLE OVERRIDEs if not used with care. For example:
* If a column is added with a table override, but then later added to the source MySQL table, the converted ALTER TABLE
query in ClickHouse will fail because the column already exists.
* It is currently possible to add overrides that reference nullable columns where not-nullable are required, such as in
`ORDER BY` or `PARTITION BY`.
## Examples of Use {#examples-of-use}

View File

@ -106,6 +106,10 @@ if (USE_AWS_S3)
add_headers_and_sources(dbms Disks/S3)
endif()
if (USE_AZURE_BLOB_STORAGE)
add_headers_and_sources(dbms Disks/BlobStorage)
endif()
if (USE_HDFS)
add_headers_and_sources(dbms Storages/HDFS)
add_headers_and_sources(dbms Disks/HDFS)
@ -450,6 +454,11 @@ if (USE_AWS_S3)
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${AWS_S3_INCLUDE_DIR})
endif()
if (USE_AZURE_BLOB_STORAGE)
target_link_libraries (clickhouse_common_io PUBLIC ${AZURE_BLOB_STORAGE_LIBRARY})
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${AZURE_SDK_INCLUDES})
endif()
if (USE_S2_GEOMETRY)
dbms_target_link_libraries (PUBLIC ${S2_GEOMETRY_LIBRARY})
dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${S2_GEOMETRY_INCLUDE_DIR})

View File

@ -470,6 +470,7 @@
M(497, ACCESS_DENIED) \
M(498, LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED) \
M(499, S3_ERROR) \
M(500, BLOB_STORAGE_ERROR) \
M(501, CANNOT_CREATE_DATABASE) \
M(502, CANNOT_SIGQUEUE) \
M(503, AGGREGATE_FUNCTION_THROW) \

View File

@ -9,6 +9,7 @@
#cmakedefine01 USE_HDFS
#cmakedefine01 USE_INTERNAL_HDFS3_LIBRARY
#cmakedefine01 USE_AWS_S3
#cmakedefine01 USE_AZURE_BLOB_STORAGE
#cmakedefine01 USE_BROTLI
#cmakedefine01 USE_UNWIND
#cmakedefine01 USE_OPENCL

View File

@ -1,401 +0,0 @@
#include <DataStreams/ConvertingBlockInputStream.h>
#include <DataStreams/PushingToViewsBlockOutputStream.h>
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTInsertQuery.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <Common/ThreadPool.h>
#include <Common/checkStackSize.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/StorageValues.h>
#include <Storages/WindowView/StorageWindowView.h>
#include <Storages/LiveView/StorageLiveView.h>
#include <Storages/StorageMaterializedView.h>
#include <common/logger_useful.h>
namespace DB
{
PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
const StoragePtr & storage_,
const StorageMetadataPtr & metadata_snapshot_,
ContextPtr context_,
const ASTPtr & query_ptr_,
bool no_destination)
: WithContext(context_)
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, log(&Poco::Logger::get("PushingToViewsBlockOutputStream"))
, query_ptr(query_ptr_)
{
checkStackSize();
/** TODO This is a very important line. At any insertion into the table one of streams should own lock.
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
* but it's clear that here is not the best place for this functionality.
*/
addTableLock(
storage->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout));
/// If the "root" table deduplicates blocks, there are no need to make deduplication for children
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
bool disable_deduplication_for_children = false;
if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
disable_deduplication_for_children = !no_destination && storage->supportsDeduplication();
auto table_id = storage->getStorageID();
Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id);
/// We need special context for materialized views insertions
if (!dependencies.empty())
{
select_context = Context::createCopy(context);
insert_context = Context::createCopy(context);
const auto & insert_settings = insert_context->getSettingsRef();
// Do not deduplicate insertions into MV if the main insertion is Ok
if (disable_deduplication_for_children)
insert_context->setSetting("insert_deduplicate", Field{false});
// Separate min_insert_block_size_rows/min_insert_block_size_bytes for children
if (insert_settings.min_insert_block_size_rows_for_materialized_views)
insert_context->setSetting("min_insert_block_size_rows", insert_settings.min_insert_block_size_rows_for_materialized_views.value);
if (insert_settings.min_insert_block_size_bytes_for_materialized_views)
insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value);
}
for (const auto & database_table : dependencies)
{
auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext());
auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr();
ASTPtr query;
BlockOutputStreamPtr out;
if (auto * materialized_view = dynamic_cast<StorageMaterializedView *>(dependent_table.get()))
{
addTableLock(
materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout));
StoragePtr inner_table = materialized_view->getTargetTable();
auto inner_table_id = inner_table->getStorageID();
auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr();
query = dependent_metadata_snapshot->getSelectQuery().inner_query;
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
insert->table_id = inner_table_id;
/// Get list of columns we get from select query.
auto header = InterpreterSelectQuery(query, select_context, SelectQueryOptions().analyze())
.getSampleBlock();
/// Insert only columns returned by select.
auto list = std::make_shared<ASTExpressionList>();
const auto & inner_table_columns = inner_metadata_snapshot->getColumns();
for (const auto & column : header)
{
/// But skip columns which storage doesn't have.
if (inner_table_columns.hasPhysical(column.name))
list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
}
insert->columns = std::move(list);
ASTPtr insert_query_ptr(insert.release());
InterpreterInsertQuery interpreter(insert_query_ptr, insert_context);
BlockIO io = interpreter.execute();
out = io.out;
}
else if (
dynamic_cast<const StorageLiveView *>(dependent_table.get()) || dynamic_cast<const StorageWindowView *>(dependent_table.get()))
out = std::make_shared<PushingToViewsBlockOutputStream>(
dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true);
else
out = std::make_shared<PushingToViewsBlockOutputStream>(
dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr());
views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, 0 /* elapsed_ms */});
}
/// Do not push to destination table if the flag is set
if (!no_destination)
{
output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), getContext());
replicated_output = dynamic_cast<ReplicatedMergeTreeBlockOutputStream *>(output.get());
}
}
Block PushingToViewsBlockOutputStream::getHeader() const
{
/// If we don't write directly to the destination
/// then expect that we're inserting with precalculated virtual columns
if (output)
return metadata_snapshot->getSampleBlock();
else
return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals());
}
void PushingToViewsBlockOutputStream::write(const Block & block)
{
/** Throw an exception if the sizes of arrays - elements of nested data structures doesn't match.
* We have to make this assertion before writing to table, because storage engine may assume that they have equal sizes.
* NOTE It'd better to do this check in serialization of nested structures (in place when this assumption is required),
* but currently we don't have methods for serialization of nested structures "as a whole".
*/
Nested::validateArraySizes(block);
if (auto * live_view = dynamic_cast<StorageLiveView *>(storage.get()))
{
StorageLiveView::writeIntoLiveView(*live_view, block, getContext());
}
else if (auto * window_view = dynamic_cast<StorageWindowView *>(storage.get()))
{
StorageWindowView::writeIntoWindowView(*window_view, block, getContext());
}
else
{
if (output)
/// TODO: to support virtual and alias columns inside MVs, we should return here the inserted block extended
/// with additional columns directly from storage and pass it to MVs instead of raw block.
output->write(block);
}
/// Don't process materialized views if this block is duplicate
if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate())
return;
// Insert data into materialized views only after successful insert into main table
const Settings & settings = getContext()->getSettingsRef();
if (settings.parallel_view_processing && views.size() > 1)
{
// Push to views concurrently if enabled and more than one view is attached
ThreadPool pool(std::min(size_t(settings.max_threads), views.size()));
for (auto & view : views)
{
auto thread_group = CurrentThread::getGroup();
pool.scheduleOrThrowOnError([=, &view, this]
{
setThreadName("PushingToViews");
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
process(block, view);
});
}
// Wait for concurrent view processing
pool.wait();
}
else
{
// Process sequentially
for (auto & view : views)
{
process(block, view);
if (view.exception)
std::rethrow_exception(view.exception);
}
}
}
void PushingToViewsBlockOutputStream::writePrefix()
{
if (output)
output->writePrefix();
for (auto & view : views)
{
try
{
view.out->writePrefix();
}
catch (Exception & ex)
{
ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs());
throw;
}
}
}
void PushingToViewsBlockOutputStream::writeSuffix()
{
if (output)
output->writeSuffix();
std::exception_ptr first_exception;
const Settings & settings = getContext()->getSettingsRef();
bool parallel_processing = false;
/// Run writeSuffix() for views in separate thread pool.
/// In could have been done in PushingToViewsBlockOutputStream::process, however
/// it is not good if insert into main table fail but into view succeed.
if (settings.parallel_view_processing && views.size() > 1)
{
parallel_processing = true;
// Push to views concurrently if enabled and more than one view is attached
ThreadPool pool(std::min(size_t(settings.max_threads), views.size()));
auto thread_group = CurrentThread::getGroup();
for (auto & view : views)
{
if (view.exception)
continue;
pool.scheduleOrThrowOnError([thread_group, &view, this]
{
setThreadName("PushingToViews");
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
Stopwatch watch;
try
{
view.out->writeSuffix();
}
catch (...)
{
view.exception = std::current_exception();
}
view.elapsed_ms += watch.elapsedMilliseconds();
LOG_TRACE(log, "Pushing from {} to {} took {} ms.",
storage->getStorageID().getNameForLogs(),
view.table_id.getNameForLogs(),
view.elapsed_ms);
});
}
// Wait for concurrent view processing
pool.wait();
}
for (auto & view : views)
{
if (view.exception)
{
if (!first_exception)
first_exception = view.exception;
continue;
}
if (parallel_processing)
continue;
Stopwatch watch;
try
{
view.out->writeSuffix();
}
catch (Exception & ex)
{
ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs());
throw;
}
view.elapsed_ms += watch.elapsedMilliseconds();
LOG_TRACE(log, "Pushing from {} to {} took {} ms.",
storage->getStorageID().getNameForLogs(),
view.table_id.getNameForLogs(),
view.elapsed_ms);
}
if (first_exception)
std::rethrow_exception(first_exception);
UInt64 milliseconds = main_watch.elapsedMilliseconds();
if (views.size() > 1)
{
LOG_DEBUG(log, "Pushing from {} to {} views took {} ms.",
storage->getStorageID().getNameForLogs(), views.size(),
milliseconds);
}
}
void PushingToViewsBlockOutputStream::flush()
{
if (output)
output->flush();
for (auto & view : views)
view.out->flush();
}
void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & view)
{
Stopwatch watch;
try
{
BlockInputStreamPtr in;
/// We need keep InterpreterSelectQuery, until the processing will be finished, since:
///
/// - We copy Context inside InterpreterSelectQuery to support
/// modification of context (Settings) for subqueries
/// - InterpreterSelectQuery lives shorter than query pipeline.
/// It's used just to build the query pipeline and no longer needed
/// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery,
/// **can** take a reference to Context from InterpreterSelectQuery
/// (the problem raises only when function uses context from the
/// execute*() method, like FunctionDictGet do)
/// - These objects live inside query pipeline (DataStreams) and the reference become dangling.
std::optional<InterpreterSelectQuery> select;
if (view.query)
{
/// We create a table with the same name as original table and the same alias columns,
/// but it will contain single block (that is INSERT-ed into main table).
/// InterpreterSelectQuery will do processing of alias columns.
auto local_context = Context::createCopy(select_context);
local_context->addViewSource(
StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals()));
select.emplace(view.query, local_context, SelectQueryOptions());
in = std::make_shared<MaterializingBlockInputStream>(select->execute().getInputStream());
/// Squashing is needed here because the materialized view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
in = std::make_shared<SquashingBlockInputStream>(
in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes);
in = std::make_shared<ConvertingBlockInputStream>(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name);
}
else
in = std::make_shared<OneBlockInputStream>(block);
in->readPrefix();
while (Block result_block = in->read())
{
Nested::validateArraySizes(result_block);
view.out->write(result_block);
}
in->readSuffix();
}
catch (Exception & ex)
{
ex.addMessage("while pushing to view " + view.table_id.getNameForLogs());
view.exception = std::current_exception();
}
catch (...)
{
view.exception = std::current_exception();
}
view.elapsed_ms += watch.elapsedMilliseconds();
}
}

View File

@ -117,6 +117,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
"Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"};
static const std::unordered_set<std::string_view> engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL"};
bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
if (engine_define->engine->arguments && !engine_may_have_arguments)
@ -131,6 +132,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST,
"Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name);
if (create.table_overrides && !engines_with_table_overrides.contains(engine_name))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name);
if (engine_name == "Ordinary")
return std::make_shared<DatabaseOrdinary>(database_name, metadata_path, context);
else if (engine_name == "Atomic")

View File

@ -9,7 +9,6 @@
# include <Databases/MySQL/MaterializedMySQLSyncThread.h>
# include <Parsers/ASTCreateQuery.h>
# include <Storages/StorageMaterializedMySQL.h>
# include <Poco/Logger.h>
# include <Common/setThreadName.h>
# include <filesystem>

View File

@ -0,0 +1,145 @@
#include <Disks/BlobStorage/BlobStorageAuth.h>
#if USE_AZURE_BLOB_STORAGE
#include <optional>
#include <re2/re2.h>
#include <azure/identity/managed_identity_credential.hpp>
using namespace Azure::Storage::Blobs;
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
struct BlobStorageEndpoint
{
const String storage_account_url;
const String container_name;
const std::optional<bool> container_already_exists;
};
void validateStorageAccountUrl(const String & storage_account_url)
{
const auto * storage_account_url_pattern_str = R"(http(()|s)://[a-z0-9-.:]+(()|/)[a-z0-9]*(()|/))";
static const RE2 storage_account_url_pattern(storage_account_url_pattern_str);
if (!re2::RE2::FullMatch(storage_account_url, storage_account_url_pattern))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Blob Storage URL is not valid, should follow the format: {}, got: {}", storage_account_url_pattern_str, storage_account_url);
}
void validateContainerName(const String & container_name)
{
auto len = container_name.length();
if (len < 3 || len > 64)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Blob Storage container name is not valid, should have length between 3 and 64, but has length: {}", len);
const auto * container_name_pattern_str = R"([a-z][a-z0-9-]+)";
static const RE2 container_name_pattern(container_name_pattern_str);
if (!re2::RE2::FullMatch(container_name, container_name_pattern))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Blob Storage container name is not valid, should follow the format: {}, got: {}", container_name_pattern_str, container_name);
}
BlobStorageEndpoint processBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
String storage_account_url = config.getString(config_prefix + ".storage_account_url");
validateStorageAccountUrl(storage_account_url);
String container_name = config.getString(config_prefix + ".container_name", "default-container");
validateContainerName(container_name);
std::optional<bool> container_already_exists {};
if (config.has(config_prefix + ".container_already_exists"))
container_already_exists = {config.getBool(config_prefix + ".container_already_exists")};
return {storage_account_url, container_name, container_already_exists};
}
template <class T>
std::shared_ptr<T> getClientWithConnectionString(const String & connection_str, const String & container_name) = delete;
template<>
std::shared_ptr<BlobServiceClient> getClientWithConnectionString(
const String & connection_str, const String & /*container_name*/)
{
return std::make_shared<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(connection_str));
}
template<>
std::shared_ptr<BlobContainerClient> getClientWithConnectionString(
const String & connection_str, const String & container_name)
{
return std::make_shared<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(connection_str, container_name));
}
template <class T>
std::shared_ptr<T> getBlobStorageClientWithAuth(
const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
if (config.has(config_prefix + ".connection_string"))
{
String connection_str = config.getString(config_prefix + ".connection_string");
return getClientWithConnectionString<T>(connection_str, container_name);
}
if (config.has(config_prefix + ".account_key") && config.has(config_prefix + ".account_name"))
{
auto storage_shared_key_credential = std::make_shared<Azure::Storage::StorageSharedKeyCredential>(
config.getString(config_prefix + ".account_name"),
config.getString(config_prefix + ".account_key")
);
return std::make_shared<T>(url, storage_shared_key_credential);
}
auto managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
return std::make_shared<T>(url, managed_identity_credential);
}
std::shared_ptr<BlobContainerClient> getBlobContainerClient(
const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
auto endpoint = processBlobStorageEndpoint(config, config_prefix);
auto container_name = endpoint.container_name;
auto final_url = endpoint.storage_account_url
+ (endpoint.storage_account_url.back() == '/' ? "" : "/")
+ container_name;
if (endpoint.container_already_exists.value_or(false))
return getBlobStorageClientWithAuth<BlobContainerClient>(final_url, container_name, config, config_prefix);
auto blob_service_client = getBlobStorageClientWithAuth<BlobServiceClient>(endpoint.storage_account_url, container_name, config, config_prefix);
if (!endpoint.container_already_exists.has_value())
{
ListBlobContainersOptions blob_containers_list_options;
blob_containers_list_options.Prefix = container_name;
blob_containers_list_options.PageSizeHint = 1;
auto blob_containers = blob_service_client->ListBlobContainers().BlobContainers;
for (const auto & blob_container : blob_containers)
{
if (blob_container.Name == endpoint.container_name)
return getBlobStorageClientWithAuth<BlobContainerClient>(final_url, container_name, config, config_prefix);
}
}
return std::make_shared<BlobContainerClient>(
blob_service_client->CreateBlobContainer(container_name).Value);
}
}
#endif

View File

@ -0,0 +1,20 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <Disks/IDiskRemote.h>
#include <azure/storage/blobs.hpp>
namespace DB
{
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> getBlobContainerClient(
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
}
#endif

View File

@ -0,0 +1,190 @@
#include <Disks/BlobStorage/DiskBlobStorage.h>
#if USE_AZURE_BLOB_STORAGE
#include <Disks/RemoteDisksCommon.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BLOB_STORAGE_ERROR;
}
DiskBlobStorageSettings::DiskBlobStorageSettings(
UInt64 max_single_part_upload_size_,
UInt64 min_bytes_for_seek_,
int max_single_read_retries_,
int max_single_download_retries_,
int thread_pool_size_) :
max_single_part_upload_size(max_single_part_upload_size_),
min_bytes_for_seek(min_bytes_for_seek_),
max_single_read_retries(max_single_read_retries_),
max_single_download_retries(max_single_download_retries_),
thread_pool_size(thread_pool_size_) {}
class BlobStoragePathKeeper : public RemoteFSPathKeeper
{
public:
/// RemoteFSPathKeeper constructed with a placeholder argument for chunk_limit, it is unused in this class
BlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {}
void addPath(const String & path) override
{
paths.push_back(path);
}
std::vector<String> paths;
};
DiskBlobStorage::DiskBlobStorage(
const String & name_,
DiskPtr metadata_disk_,
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
SettingsPtr settings_,
GetDiskSettings settings_getter_) :
IDiskRemote(name_, "", metadata_disk_, "DiskBlobStorage", settings_->thread_pool_size),
blob_container_client(blob_container_client_),
current_settings(std::move(settings_)),
settings_getter(settings_getter_) {}
std::unique_ptr<ReadBufferFromFileBase> DiskBlobStorage::readFile(
const String & path,
const ReadSettings & read_settings,
std::optional<size_t> /*estimated_size*/) const
{
auto settings = current_settings.get();
auto metadata = readMeta(path);
LOG_TRACE(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path));
bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool;
auto reader_impl = std::make_unique<ReadBufferFromBlobStorageGather>(
path, blob_container_client, metadata, settings->max_single_read_retries,
settings->max_single_download_retries, read_settings, threadpool_read);
if (threadpool_read)
{
auto reader = getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(reader_impl));
}
else
{
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(reader_impl));
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), current_settings.get()->min_bytes_for_seek);
}
}
std::unique_ptr<WriteBufferFromFileBase> DiskBlobStorage::writeFile(
const String & path,
size_t buf_size,
WriteMode mode)
{
auto metadata = readOrCreateMetaForWriting(path, mode);
auto blob_path = path + "_" + getRandomName(8); /// NOTE: path contains the tmp_* prefix in the blob name
LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}",
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path);
auto buffer = std::make_unique<WriteBufferFromBlobStorage>(
blob_container_client,
blob_path,
current_settings.get()->max_single_part_upload_size,
buf_size);
return std::make_unique<WriteIndirectBufferFromRemoteFS<WriteBufferFromBlobStorage>>(std::move(buffer), std::move(metadata), blob_path);
}
DiskType DiskBlobStorage::getType() const
{
return DiskType::BlobStorage;
}
bool DiskBlobStorage::isRemote() const
{
return true;
}
bool DiskBlobStorage::supportZeroCopyReplication() const
{
return true;
}
bool DiskBlobStorage::checkUniqueId(const String & id) const
{
Azure::Storage::Blobs::ListBlobsOptions blobs_list_options;
blobs_list_options.Prefix = id;
blobs_list_options.PageSizeHint = 1;
auto blobs_list_response = blob_container_client->ListBlobs(blobs_list_options);
auto blobs_list = blobs_list_response.Blobs;
for (const auto & blob : blobs_list)
{
if (id == blob.Name)
return true;
}
return false;
}
void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper)
{
auto * paths_keeper = dynamic_cast<BlobStoragePathKeeper *>(fs_paths_keeper.get());
if (paths_keeper)
{
for (const auto & path : paths_keeper->paths)
{
try
{
auto delete_info = blob_container_client->DeleteBlob(path);
if (!delete_info.Value.Deleted)
throw Exception(ErrorCodes::BLOB_STORAGE_ERROR, "Failed to delete file in Blob Storage: {}", path);
}
catch (const Azure::Storage::StorageException& e)
{
LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message);
throw e;
}
}
}
}
RemoteFSPathKeeperPtr DiskBlobStorage::createFSPathKeeper() const
{
return std::make_shared<BlobStoragePathKeeper>();
}
void DiskBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &)
{
auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context);
current_settings.set(std::move(new_settings));
if (AsyncExecutor * exec = dynamic_cast<AsyncExecutor*>(&getExecutor()))
exec->setMaxThreads(current_settings.get()->thread_pool_size);
}
}
#endif

View File

@ -0,0 +1,88 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <Disks/IDiskRemote.h>
#include <IO/ReadBufferFromBlobStorage.h>
#include <IO/WriteBufferFromBlobStorage.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <azure/identity/managed_identity_credential.hpp>
#include <azure/storage/blobs.hpp>
namespace DB
{
struct DiskBlobStorageSettings final
{
DiskBlobStorageSettings(
UInt64 max_single_part_upload_size_,
UInt64 min_bytes_for_seek_,
int max_single_read_retries,
int max_single_download_retries,
int thread_pool_size_);
size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset
UInt64 min_bytes_for_seek;
size_t max_single_read_retries;
size_t max_single_download_retries;
size_t thread_pool_size;
};
class DiskBlobStorage final : public IDiskRemote
{
public:
using SettingsPtr = std::unique_ptr<DiskBlobStorageSettings>;
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextPtr)>;
DiskBlobStorage(
const String & name_,
DiskPtr metadata_disk_,
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
SettingsPtr settings_,
GetDiskSettings settings_getter_);
std::unique_ptr<ReadBufferFromFileBase> readFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> estimated_size) const override;
std::unique_ptr<WriteBufferFromFileBase> writeFile(
const String & path,
size_t buf_size,
WriteMode mode) override;
DiskType getType() const override;
bool isRemote() const override;
bool supportZeroCopyReplication() const override;
bool checkUniqueId(const String & id) const override;
void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override;
RemoteFSPathKeeperPtr createFSPathKeeper() const override;
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override;
private:
/// client used to access the files in the Blob Storage cloud
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
MultiVersion<DiskBlobStorageSettings> current_settings;
/// Gets disk settings from context.
GetDiskSettings settings_getter;
};
}
#endif

View File

@ -0,0 +1,128 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#include <Disks/DiskFactory.h>
#if USE_AZURE_BLOB_STORAGE
#include <Disks/DiskRestartProxy.h>
#include <Disks/DiskCacheWrapper.h>
#include <Disks/RemoteDisksCommon.h>
#include <Disks/BlobStorage/DiskBlobStorage.h>
#include <Disks/BlobStorage/BlobStorageAuth.h>
namespace DB
{
namespace ErrorCodes
{
extern const int PATH_ACCESS_DENIED;
}
constexpr char test_file[] = "test.txt";
constexpr char test_str[] = "test";
constexpr size_t test_str_size = 4;
void checkWriteAccess(IDisk & disk)
{
auto file = disk.writeFile(test_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
file->write(test_str, test_str_size);
}
void checkReadAccess(IDisk & disk)
{
auto file = disk.readFile(test_file);
String buf(test_str_size, '0');
file->readStrict(buf.data(), test_str_size);
if (buf != test_str)
throw Exception("No read access to disk", ErrorCodes::PATH_ACCESS_DENIED);
}
void checkReadWithOffset(IDisk & disk)
{
auto file = disk.readFile(test_file);
auto offset = 2;
auto test_size = test_str_size - offset;
String buf(test_size, '0');
file->seek(offset, 0);
file->readStrict(buf.data(), test_size);
if (buf != test_str + offset)
throw Exception("Failed to read file with offset", ErrorCodes::PATH_ACCESS_DENIED);
}
void checkRemoveAccess(IDisk & disk)
{
disk.removeFile(test_file);
}
std::unique_ptr<DiskBlobStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/)
{
return std::make_unique<DiskBlobStorageSettings>(
config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024),
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
config.getInt(config_prefix + ".max_single_read_retries", 3),
config.getInt(config_prefix + ".max_single_download_retries", 3),
config.getInt(config_prefix + ".thread_pool_size", 16)
);
}
void registerDiskBlobStorage(DiskFactory & factory)
{
auto creator = [](
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
const DisksMap & /*map*/)
{
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
std::shared_ptr<IDisk> blob_storage_disk = std::make_shared<DiskBlobStorage>(
name,
metadata_disk,
getBlobContainerClient(config, config_prefix),
getSettings(config, config_prefix, context),
getSettings
);
if (!config.getBool(config_prefix + ".skip_access_check", false))
{
checkWriteAccess(*blob_storage_disk);
checkReadAccess(*blob_storage_disk);
checkReadWithOffset(*blob_storage_disk);
checkRemoveAccess(*blob_storage_disk);
}
blob_storage_disk->startup();
if (config.getBool(config_prefix + ".cache_enabled", true))
{
String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/");
blob_storage_disk = wrapWithCache(blob_storage_disk, "blob-storage-cache", cache_path, metadata_path);
}
return std::make_shared<DiskRestartProxy>(blob_storage_disk);
};
factory.registerDiskType("blob_storage", creator);
}
}
#else
namespace DB
{
void registerDiskBlobStorage(DiskFactory &) {}
}
#endif

View File

@ -10,6 +10,7 @@ enum class DiskType
Local,
RAM,
S3,
BlobStorage,
HDFS,
Encrypted,
WebServer,
@ -25,6 +26,8 @@ inline String toString(DiskType disk_type)
return "memory";
case DiskType::S3:
return "s3";
case DiskType::BlobStorage:
return "blob_storage";
case DiskType::HDFS:
return "hdfs";
case DiskType::Encrypted:

View File

@ -1,5 +1,6 @@
#include <Disks/HDFS/DiskHDFS.h>
#include <Disks/DiskLocal.h>
#include <Disks/RemoteDisksCommon.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <Storages/HDFS/WriteBufferFromHDFS.h>
@ -160,17 +161,13 @@ void registerDiskHDFS(DiskFactory & factory)
ContextPtr context_,
const DisksMap & /*map*/) -> DiskPtr
{
fs::path disk = fs::path(context_->getPath()) / "disks" / name;
fs::create_directories(disk);
String uri{config.getString(config_prefix + ".endpoint")};
checkHDFSURL(uri);
if (uri.back() != '/')
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
String metadata_path = context_->getPath() + "disks/" + name + "/";
auto metadata_disk = std::make_shared<DiskLocal>(name + "-metadata", metadata_path, 0);
auto metadata_disk = prepareForLocalMetadata(name, config, config_prefix, context_).second;
return std::make_shared<DiskHDFS>(
name, uri,

View File

@ -42,7 +42,7 @@ class IAsynchronousReader;
using AsynchronousReaderPtr = std::shared_ptr<IAsynchronousReader>;
/// Base Disk class for remote FS's, which are not posix-compatible (DiskS3 and DiskHDFS)
/// Base Disk class for remote FS's, which are not posix-compatible (e.g. DiskS3, DiskHDFS, DiskBlobStorage)
class IDiskRemote : public IDisk
{

View File

@ -8,6 +8,10 @@
#include <IO/ReadBufferFromS3.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <IO/ReadBufferFromBlobStorage.h>
#endif
#if USE_HDFS
#include <Storages/HDFS/ReadBufferFromHDFS.h>
#endif
@ -30,6 +34,15 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S
#endif
#if USE_AZURE_BLOB_STORAGE
SeekableReadBufferPtr ReadBufferFromBlobStorageGather::createImplementationBuffer(const String & path, size_t read_until_position_) const
{
return std::make_unique<ReadBufferFromBlobStorage>(blob_container_client, path, max_single_read_retries,
max_single_download_retries, settings.remote_fs_buffer_size, threadpool_read, read_until_position_);
}
#endif
SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path, size_t read_until_position_) const
{
return std::make_unique<ReadBufferFromWebServer>(fs::path(uri) / path, context, settings, threadpool_read, read_until_position_);

View File

@ -5,6 +5,10 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadSettings.h>
#if USE_AZURE_BLOB_STORAGE
#include <azure/storage/blobs.hpp>
#endif
namespace Aws
{
namespace S3
@ -97,6 +101,40 @@ private:
#endif
#if USE_AZURE_BLOB_STORAGE
/// Reads data from Blob Storage using paths stored in metadata.
class ReadBufferFromBlobStorageGather final : public ReadBufferFromRemoteFSGather
{
public:
ReadBufferFromBlobStorageGather(
const String & path_,
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
IDiskRemote::Metadata metadata_,
size_t max_single_read_retries_,
size_t max_single_download_retries_,
const ReadSettings & settings_,
bool threadpool_read_ = false)
: ReadBufferFromRemoteFSGather(metadata_, path_)
, blob_container_client(blob_container_client_)
, max_single_read_retries(max_single_read_retries_)
, max_single_download_retries(max_single_download_retries_)
, settings(settings_)
, threadpool_read(threadpool_read_)
{
}
SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t read_until_position) const override;
private:
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
size_t max_single_read_retries;
size_t max_single_download_retries;
ReadSettings settings;
bool threadpool_read;
};
#endif
class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather
{
public:

View File

@ -1,6 +1,7 @@
#include "WriteIndirectBufferFromRemoteFS.h"
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteBufferFromBlobStorage.h>
#include <Storages/HDFS/WriteBufferFromHDFS.h>
#include <IO/WriteBufferFromHTTP.h>
@ -57,6 +58,11 @@ template
class WriteIndirectBufferFromRemoteFS<WriteBufferFromS3>;
#endif
#if USE_AZURE_BLOB_STORAGE
template
class WriteIndirectBufferFromRemoteFS<WriteBufferFromBlobStorage>;
#endif
#if USE_HDFS
template
class WriteIndirectBufferFromRemoteFS<WriteBufferFromHDFS>;

View File

@ -0,0 +1,53 @@
#include <Disks/RemoteDisksCommon.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
String getRandomName(size_t len, char first, char last)
{
std::uniform_int_distribution<int> distribution(first, last);
String res(len, ' ');
for (auto & c : res)
c = distribution(thread_local_rng);
return res;
}
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path)
{
if (metadata_path == cache_path)
throw Exception("Metadata and cache paths should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS);
auto cache_disk = std::make_shared<DiskLocal>(cache_name, cache_path, 0);
auto cache_file_predicate = [] (const String & path)
{
return path.ends_with("idx") // index files.
|| path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") /// mark files.
|| path.ends_with("txt") || path.ends_with("dat");
};
return std::make_shared<DiskCacheWrapper>(disk, cache_disk, cache_file_predicate);
}
std::pair<String, DiskPtr> prepareForLocalMetadata(
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context)
{
/// where the metadata files are stored locally
auto metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/");
fs::create_directories(metadata_path);
auto metadata_disk = std::make_shared<DiskLocal>(name + "-metadata", metadata_path, 0);
return std::make_pair(metadata_path, metadata_disk);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <random>
#include <utility>
#include <Core/Types.h>
#include <Common/thread_local_rng.h>
#include <Disks/IDisk.h>
#include <Disks/DiskCacheWrapper.h>
namespace DB
{
String getRandomName(size_t len = 32, char first = 'a', char last = 'z');
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path);
std::pair<String, DiskPtr> prepareForLocalMetadata(
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context);
}

View File

@ -25,6 +25,7 @@
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
#include <Disks/RemoteDisksCommon.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
@ -100,15 +101,6 @@ private:
Chunks chunks;
};
String getRandomName()
{
std::uniform_int_distribution<int> distribution('a', 'z');
String res(32, ' '); /// The number of bits of entropy should be not less than 128.
for (auto & c : res)
c = distribution(thread_local_rng);
return res;
}
template <typename Result, typename Error>
void throwIfError(Aws::Utils::Outcome<Result, Error> & response)
{

View File

@ -18,6 +18,7 @@
#include "ProxyResolverConfiguration.h"
#include "Disks/DiskRestartProxy.h"
#include "Disks/DiskLocal.h"
#include "Disks/RemoteDisksCommon.h"
namespace DB
{
@ -176,9 +177,7 @@ void registerDiskS3(DiskFactory & factory)
if (uri.key.back() != '/')
throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS);
String metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/");
fs::create_directories(metadata_path);
auto metadata_disk = std::make_shared<DiskLocal>(name + "-metadata", metadata_path, 0);
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
std::shared_ptr<IDisk> s3disk = std::make_shared<DiskS3>(
name,
@ -199,24 +198,10 @@ void registerDiskS3(DiskFactory & factory)
s3disk->startup();
bool cache_enabled = config.getBool(config_prefix + ".cache_enabled", true);
if (cache_enabled)
if (config.getBool(config_prefix + ".cache_enabled", true))
{
String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/");
if (metadata_path == cache_path)
throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS);
auto cache_disk = std::make_shared<DiskLocal>("s3-cache", cache_path, 0);
auto cache_file_predicate = [] (const String & path)
{
return path.ends_with("idx") // index files.
|| path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") // mark files.
|| path.ends_with("txt") || path.ends_with("dat");
};
s3disk = std::make_shared<DiskCacheWrapper>(s3disk, cache_disk, cache_file_predicate);
s3disk = wrapWithCache(s3disk, "s3-cache", cache_path, metadata_path);
}
return std::make_shared<DiskRestartProxy>(s3disk);

View File

@ -14,6 +14,10 @@ void registerDiskMemory(DiskFactory & factory);
void registerDiskS3(DiskFactory & factory);
#endif
#if USE_AZURE_BLOB_STORAGE
void registerDiskBlobStorage(DiskFactory & factory);
#endif
#if USE_SSL
void registerDiskEncrypted(DiskFactory & factory);
#endif
@ -36,6 +40,10 @@ void registerDisks()
registerDiskS3(factory);
#endif
#if USE_AZURE_BLOB_STORAGE
registerDiskBlobStorage(factory);
#endif
#if USE_SSL
registerDiskEncrypted(factory);
#endif

View File

@ -860,7 +860,7 @@ namespace
template <typename NumberType>
void toStringAppend(NumberType value, PaddedPODArray<UInt8> & str)
{
WriteBufferFromVector buf{str, WriteBufferFromVector<PaddedPODArray<UInt8>>::AppendModeTag{}};
WriteBufferFromVector buf{str, AppendModeTag{}};
writeText(value, buf);
}

View File

@ -3,10 +3,13 @@
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/copyData.h>
#include <Interpreters/Context.h>
#include <unistd.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
@ -16,9 +19,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_FILE_NAME;
extern const int DATABASE_ACCESS_DENIED;
extern const int FILE_DOESNT_EXIST;
}
/// A function to read file as a string.
@ -30,15 +31,14 @@ public:
explicit FunctionFile(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (!isString(arguments[0].type))
throw Exception(getName() + " is only implemented for types String", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is only implemented for type String", getName());
return std::make_shared<DataTypeString>();
}
@ -47,80 +47,50 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const ColumnPtr column = arguments[0].column;
const ColumnString * expected = checkAndGetColumn<ColumnString>(column.get());
if (!expected)
const ColumnString * column_src = checkAndGetColumn<ColumnString>(column.get());
if (!column_src)
throw Exception(
fmt::format("Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()),
ErrorCodes::ILLEGAL_COLUMN);
const ColumnString::Chars & chars = expected->getChars();
const ColumnString::Offsets & offsets = expected->getOffsets();
std::vector<String> checked_filenames(input_rows_count);
auto result = ColumnString::create();
auto & res_chars = result->getChars();
auto & res_offsets = result->getOffsets();
res_offsets.resize(input_rows_count);
size_t source_offset = 0;
size_t result_offset = 0;
fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath()));
std::string user_files_absolute_path_string = user_files_absolute_path.string();
// If run in Local mode, no need for path checking.
bool need_check = getContext()->getApplicationType() != Context::ApplicationType::LOCAL;
for (size_t row = 0; row < input_rows_count; ++row)
{
const char * filename = reinterpret_cast<const char *>(&chars[source_offset]);
StringRef filename = column_src->getDataAt(row);
fs::path file_path(filename.data, filename.data + filename.size);
fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath()));
fs::path file_path(filename);
if (file_path.is_relative())
file_path = user_files_absolute_path / file_path;
fs::path file_absolute_path = fs::canonical(file_path);
checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path);
checked_filenames[row] = file_absolute_path.string();
/// Do not use fs::canonical or fs::weakly_canonical.
/// Otherwise it will not allow to work with symlinks in `user_files_path` directory.
file_path = fs::absolute(file_path).lexically_normal();
if (!fs::exists(file_absolute_path))
throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path.string()), ErrorCodes::FILE_DOESNT_EXIST);
if (need_check && file_path.string().find(user_files_absolute_path_string) != 0)
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_absolute_path.string());
const auto current_file_size = fs::file_size(file_absolute_path);
ReadBufferFromFile in(file_path);
WriteBufferFromVector out(res_chars, AppendModeTag{});
copyData(in, out);
out.finalize();
result_offset += current_file_size + 1;
res_offsets[row] = result_offset;
source_offset = offsets[row];
}
res_chars.resize(result_offset);
size_t prev_offset = 0;
for (size_t row = 0; row < input_rows_count; ++row)
{
auto file_absolute_path = checked_filenames[row];
ReadBufferFromFile in(file_absolute_path);
char * res_buf = reinterpret_cast<char *>(&res_chars[prev_offset]);
const size_t file_lenght = res_offsets[row] - prev_offset - 1;
prev_offset = res_offsets[row];
in.readStrict(res_buf, file_lenght);
res_buf[file_lenght] = '\0';
res_chars.push_back(0);
res_offsets[row] = res_chars.size();
}
return result;
}
private:
void checkReadIsAllowedOrThrow(const std::string & user_files_absolute_path, const std::string & file_absolute_path) const
{
// If run in Local mode, no need for path checking.
if (getContext()->getApplicationType() != Context::ApplicationType::LOCAL)
if (file_absolute_path.find(user_files_absolute_path) != 0)
throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED);
fs::path fs_path(file_absolute_path);
if (fs::exists(fs_path) && fs::is_directory(fs_path))
throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME);
}
};

View File

@ -1249,7 +1249,7 @@ public:
{
ColumnString & col_str = assert_cast<ColumnString &>(dest);
auto & chars = col_str.getChars();
WriteBufferFromVector<ColumnString::Chars> buf(chars, WriteBufferFromVector<ColumnString::Chars>::AppendModeTag());
WriteBufferFromVector<ColumnString::Chars> buf(chars, AppendModeTag());
traverse(element, buf);
buf.finalize();
chars.push_back(0);

View File

@ -0,0 +1,173 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <IO/ReadBufferFromBlobStorage.h>
#include <IO/ReadBufferFromString.h>
#include <base/logger_useful.h>
#include <base/sleep.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int SEEK_POSITION_OUT_OF_BOUND;
extern const int RECEIVED_EMPTY_DATA;
extern const int LOGICAL_ERROR;
}
ReadBufferFromBlobStorage::ReadBufferFromBlobStorage(
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
const String & path_,
size_t max_single_read_retries_,
size_t max_single_download_retries_,
size_t tmp_buffer_size_,
bool use_external_buffer_,
size_t read_until_position_)
: SeekableReadBuffer(nullptr, 0)
, blob_container_client(blob_container_client_)
, path(path_)
, max_single_read_retries(max_single_read_retries_)
, max_single_download_retries(max_single_download_retries_)
, tmp_buffer_size(tmp_buffer_size_)
, use_external_buffer(use_external_buffer_)
, read_until_position(read_until_position_)
{
if (!use_external_buffer)
{
tmp_buffer.resize(tmp_buffer_size);
data_ptr = tmp_buffer.data();
data_capacity = tmp_buffer_size;
}
}
bool ReadBufferFromBlobStorage::nextImpl()
{
if (read_until_position)
{
if (read_until_position == offset)
return false;
if (read_until_position < offset)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1);
}
if (!initialized)
initialize();
if (use_external_buffer)
{
data_ptr = internal_buffer.begin();
data_capacity = internal_buffer.size();
}
size_t to_read_bytes = std::min(total_size - offset, data_capacity);
size_t bytes_read = 0;
size_t sleep_time_with_backoff_milliseconds = 100;
for (size_t i = 0; i < max_single_read_retries; ++i)
{
try
{
bytes_read = data_stream->ReadToCount(reinterpret_cast<uint8_t *>(data_ptr), to_read_bytes);
break;
}
catch (const Azure::Storage::StorageException & e)
{
LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {} : {}", path, i, e.Message);
if (i + 1 == max_single_read_retries)
throw e;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
initialized = false;
initialize();
}
}
if (bytes_read == 0)
return false;
BufferBase::set(data_ptr, bytes_read, 0);
offset += bytes_read;
return true;
}
off_t ReadBufferFromBlobStorage::seek(off_t offset_, int whence)
{
if (initialized)
throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
if (whence != SEEK_SET)
throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
if (offset_ < 0)
throw Exception("Seek position is out of bounds. Offset: " + std::to_string(offset_), ErrorCodes::SEEK_POSITION_OUT_OF_BOUND);
offset = offset_;
return offset;
}
off_t ReadBufferFromBlobStorage::getPosition()
{
return offset - available();
}
void ReadBufferFromBlobStorage::initialize()
{
if (initialized)
return;
Azure::Storage::Blobs::DownloadBlobOptions download_options;
Azure::Nullable<int64_t> length {};
if (read_until_position != 0)
length = {static_cast<int64_t>(read_until_position - offset)};
download_options.Range = {static_cast<int64_t>(offset), length};
blob_client = std::make_unique<Azure::Storage::Blobs::BlobClient>(blob_container_client->GetBlobClient(path));
size_t sleep_time_with_backoff_milliseconds = 100;
for (size_t i = 0; i < max_single_download_retries; ++i)
{
try
{
auto download_response = blob_client->Download(download_options);
data_stream = std::move(download_response.Value.BodyStream);
break;
}
catch (const Azure::Storage::StorageException & e)
{
LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i, e.Message);
if (i + 1 == max_single_download_retries)
throw e;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
}
}
if (data_stream == nullptr)
throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Null data stream obtained while downloading file {} from Blob Storage", path);
total_size = data_stream->Length() + offset;
initialized = true;
}
}
#endif

View File

@ -0,0 +1,63 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <IO/HTTPCommon.h>
#include <IO/SeekableReadBuffer.h>
#include <IO/ReadSettings.h>
#include <azure/storage/blobs.hpp>
namespace DB
{
class ReadBufferFromBlobStorage : public SeekableReadBuffer
{
public:
explicit ReadBufferFromBlobStorage(
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
const String & path_,
size_t max_single_read_retries_,
size_t max_single_download_retries_,
size_t tmp_buffer_size_,
bool use_external_buffer_ = false,
size_t read_until_position_ = 0
);
off_t seek(off_t off, int whence) override;
off_t getPosition() override;
bool nextImpl() override;
private:
void initialize();
std::unique_ptr<Azure::Core::IO::BodyStream> data_stream;
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
std::unique_ptr<Azure::Storage::Blobs::BlobClient> blob_client;
const String path;
size_t max_single_read_retries;
size_t max_single_download_retries;
std::vector<char> tmp_buffer;
size_t tmp_buffer_size;
bool use_external_buffer;
off_t read_until_position = 0;
off_t offset = 0;
size_t total_size;
bool initialized = false;
char * data_ptr;
size_t data_capacity;
Poco::Logger * log = &Poco::Logger::get("ReadBufferFromBlobStorage");
};
}
#endif

View File

@ -0,0 +1,67 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <IO/WriteBufferFromBlobStorage.h>
#include <Disks/RemoteDisksCommon.h>
namespace DB
{
WriteBufferFromBlobStorage::WriteBufferFromBlobStorage(
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
const String & blob_path_,
size_t max_single_part_upload_size_,
size_t buf_size_) :
BufferWithOwnMemory<WriteBuffer>(buf_size_, nullptr, 0),
blob_container_client(blob_container_client_),
max_single_part_upload_size(max_single_part_upload_size_),
blob_path(blob_path_) {}
WriteBufferFromBlobStorage::~WriteBufferFromBlobStorage()
{
finalize();
}
void WriteBufferFromBlobStorage::nextImpl()
{
if (!offset())
return;
auto * buffer_begin = working_buffer.begin();
auto len = offset();
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
size_t read = 0;
while (read < len)
{
auto part_len = std::min(len - read, max_single_part_upload_size);
auto block_id = getRandomName(64);
block_ids.push_back(block_id);
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + read), part_len);
block_blob_client.StageBlock(block_id, tmp_buffer);
read += part_len;
}
}
void WriteBufferFromBlobStorage::finalizeImpl()
{
next();
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
block_blob_client.CommitBlockList(block_ids);
finalized = true;
}
}
#endif

View File

@ -0,0 +1,46 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AZURE_BLOB_STORAGE
#include <memory>
#include <IO/BufferWithOwnMemory.h>
#include <IO/WriteBuffer.h>
#include <azure/storage/blobs.hpp>
#include <azure/core/io/body_stream.hpp>
namespace DB
{
class WriteBufferFromBlobStorage : public BufferWithOwnMemory<WriteBuffer>
{
public:
explicit WriteBufferFromBlobStorage(
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
const String & blob_path_,
size_t max_single_part_upload_size_,
size_t buf_size_);
~WriteBufferFromBlobStorage() override;
void nextImpl() override;
private:
void finalizeImpl() override;
std::vector<std::string> block_ids;
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
size_t max_single_part_upload_size;
const String blob_path;
};
}
#endif

View File

@ -13,6 +13,8 @@ namespace ErrorCodes
extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER;
}
struct AppendModeTag {};
/** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size.
*
* In destructor, vector is cut to the size of written data.
@ -35,7 +37,6 @@ public:
}
/// Append to vector instead of rewrite.
struct AppendModeTag {};
WriteBufferFromVector(VectorType & vector_, AppendModeTag)
: WriteBuffer(nullptr, 0), vector(vector_)
{

View File

@ -68,8 +68,13 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac
template <typename F>
static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffer & buf, F && write_quoted_string)
{
if (isValidIdentifier(std::string_view{s}))
if (isValidIdentifier(std::string_view{s})
/// This are valid identifiers but are problematic if present unquoted in SQL query.
&& !(s.size == strlen("distinct") && 0 == strncasecmp(s.data, "distinct", strlen("distinct")))
&& !(s.size == strlen("all") && 0 == strncasecmp(s.data, "all", strlen("all"))))
{
writeString(s, buf);
}
else
write_quoted_string(s, buf);
}

View File

@ -77,7 +77,14 @@ void ZlibDeflatingWriteBuffer::nextImpl()
ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer()
{
finalize();
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void ZlibDeflatingWriteBuffer::finalizeBefore()

View File

@ -34,7 +34,8 @@ class IStreamFactory;
/// - optimize_skip_unused_shards_nesting
///
/// @return new Context with adjusted settings
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
ContextMutablePtr updateSettingsForCluster(
const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
/// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read.
/// `stream_factory` object encapsulates the logic of creating streams for a different type of query

View File

@ -421,6 +421,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
NameSet updated_columns;
bool materialize_ttl_recalculate_only = materializeTTLRecalculateOnly(storage);
for (const MutationCommand & command : commands)
{
if (command.type == MutationCommand::Type::UPDATE
@ -631,7 +632,9 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
dependencies.insert(dependency);
}
}
else if (metadata_snapshot->hasRowsTTL())
else if (metadata_snapshot->hasRowsTTL()
|| metadata_snapshot->hasAnyRowsWhereTTL()
|| metadata_snapshot->hasAnyGroupByTTL())
{
for (const auto & column : all_columns)
dependencies.emplace(column.name, ColumnDependency::TTL_TARGET);

View File

@ -435,6 +435,22 @@ void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery & creat
}
}
static ASTPtr tryGetTableOverride(const String & mapped_database, const String & table)
{
if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database))
{
auto create_query = database_ptr->getCreateDatabaseQuery();
if (auto create_database_query = create_query->as<ASTCreateQuery>())
{
if (create_database_query->table_overrides)
{
return create_database_query->table_overrides->tryGetTableOverride(table);
}
}
}
return nullptr;
}
ASTs InterpreterCreateImpl::getRewrittenQueries(
const TQuery & create_query, ContextPtr context, const String & mapped_to_database, const String & mysql_database)
{
@ -519,6 +535,12 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
rewritten_query->set(rewritten_query->storage, storage);
rewritten_query->set(rewritten_query->columns_list, columns);
if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table))
{
auto override = table_override->as<ASTTableOverride>();
override->applyToCreateTableQuery(rewritten_query.get());
}
return ASTs{rewritten_query};
}

View File

@ -410,12 +410,19 @@ void optimizeDuplicateDistinct(ASTSelectQuery & select)
/// has a single argument and not an aggregate functions.
void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, ContextPtr context,
const TablesWithColumns & tables_with_columns,
const Names & sorting_key_columns)
const TreeRewriterResult & result)
{
auto order_by = select_query->orderBy();
if (!order_by)
return;
/// Do not apply optimization for Distributed and Merge storages,
/// because we can't get the sorting key of their undelying tables
/// and we can break the matching of the sorting key for `read_in_order`
/// optimization by removing monotonous functions from the prefix of key.
if (result.is_remote_storage || (result.storage && result.storage->getName() == "Merge"))
return;
for (const auto & child : order_by->children)
{
auto * order_by_element = child->as<ASTOrderByElement>();
@ -438,6 +445,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context
}
}
auto sorting_key_columns = result.metadata_snapshot ? result.metadata_snapshot->getSortingKeyColumns() : Names{};
bool is_sorting_key_prefix = true;
for (size_t i = 0; i < order_by->children.size(); ++i)
{
@ -802,8 +811,7 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
/// Replace monotonous functions with its argument
if (settings.optimize_monotonous_functions_in_order_by)
optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns,
result.metadata_snapshot ? result.metadata_snapshot->getSortingKeyColumns() : Names{});
optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns, result);
/// Remove duplicate items from ORDER BY.
/// Execute it after all order by optimizations,

View File

@ -200,6 +200,8 @@ ASTPtr ASTCreateQuery::clone() const
res->set(res->select, select->clone());
if (tables)
res->set(res->tables, tables->clone());
if (table_overrides)
res->set(res->table_overrides, table_overrides->clone());
if (dictionary)
{
@ -240,6 +242,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (storage)
storage->formatImpl(settings, state, frame);
if (table_overrides)
{
settings.ostr << settings.nl_or_ws;
table_overrides->formatImpl(settings, state, frame);
}
if (comment)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");

View File

@ -4,6 +4,7 @@
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTDictionary.h>
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
#include <Parsers/ASTTableOverrides.h>
#include <Interpreters/StorageID.h>
namespace DB
@ -79,6 +80,8 @@ public:
ASTSelectWithUnionQuery * select = nullptr;
IAST * comment = nullptr;
ASTTableOverrideList * table_overrides = nullptr; /// For CREATE DATABASE with engines that automatically create tables
bool is_dictionary{false}; /// CREATE DICTIONARY
ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)

View File

@ -0,0 +1,242 @@
#include <IO/Operators.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTTableOverrides.h>
namespace DB
{
ASTPtr ASTTableOverride::clone() const
{
auto res = std::make_shared<ASTTableOverride>(*this);
res->children.clear();
res->table_name = table_name;
if (columns)
res->set(res->columns, columns->clone());
if (storage)
res->set(res->storage, storage->clone());
return res;
}
void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState & state, FormatStateStacked frame) const
{
FormatSettings settings = settings_;
settings.always_quote_identifiers = true;
String nl_or_nothing = settings.one_line ? "" : "\n";
String nl_or_ws = settings.one_line ? " " : "\n";
String hl_keyword = settings.hilite ? hilite_keyword : "";
String hl_none = settings.hilite ? hilite_none : "";
settings.ostr << hl_keyword << "TABLE OVERRIDE " << hl_none;
ASTIdentifier(table_name).formatImpl(settings, state, frame);
if (!columns && (!storage || storage->children.empty()))
return;
auto override_frame = frame;
++override_frame.indent;
settings.ostr << nl_or_ws << '(' << nl_or_nothing;
String indent_str = settings.one_line ? "" : String(4 * override_frame.indent, ' ');
size_t override_elems = 0;
if (columns)
{
FormatStateStacked columns_frame = override_frame;
columns_frame.expression_list_always_start_on_new_line = true;
settings.ostr << indent_str << hl_keyword << "COLUMNS" << hl_none << nl_or_ws << indent_str << "(";
columns->formatImpl(settings, state, columns_frame);
settings.ostr << nl_or_nothing << indent_str << ")";
++override_elems;
}
if (storage)
{
const auto & format_storage_elem = [&](IAST * elem, const String & elem_name)
{
if (elem)
{
settings.ostr << (override_elems++ ? nl_or_ws : "")
<< indent_str
<< hl_keyword << elem_name << hl_none << ' ';
elem->formatImpl(settings, state, override_frame);
}
};
format_storage_elem(storage->partition_by, "PARTITION BY");
format_storage_elem(storage->primary_key, "PRIMARY KEY");
format_storage_elem(storage->order_by, "ORDER BY");
format_storage_elem(storage->sample_by, "SAMPLE BY");
format_storage_elem(storage->ttl_table, "TTL");
}
settings.ostr << nl_or_nothing << ')';
}
void ASTTableOverride::applyToCreateTableQuery(ASTCreateQuery * create_query) const
{
if (columns)
{
if (!create_query->columns_list)
create_query->set(create_query->columns_list, std::make_shared<ASTColumns>());
if (columns->columns)
{
for (const auto & override_column_ast : columns->columns->children)
{
auto * override_column = override_column_ast->as<ASTColumnDeclaration>();
if (!override_column)
continue;
if (!create_query->columns_list->columns)
create_query->columns_list->set(create_query->columns_list->columns, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->columns->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTColumnDeclaration>()->name == override_column->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_column_ast);
else
dest_children[exists - dest_children.begin()] = override_column_ast;
}
}
if (columns->indices)
{
for (const auto & override_index_ast : columns->indices->children)
{
auto * override_index = override_index_ast->as<ASTIndexDeclaration>();
if (!override_index)
continue;
if (!create_query->columns_list->indices)
create_query->columns_list->set(create_query->columns_list->indices, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->indices->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTIndexDeclaration>()->name == override_index->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_index_ast);
else
dest_children[exists - dest_children.begin()] = override_index_ast;
}
}
if (columns->constraints)
{
for (const auto & override_constraint_ast : columns->constraints->children)
{
auto * override_constraint = override_constraint_ast->as<ASTConstraintDeclaration>();
if (!override_constraint)
continue;
if (!create_query->columns_list->constraints)
create_query->columns_list->set(create_query->columns_list->constraints, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->constraints->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTConstraintDeclaration>()->name == override_constraint->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_constraint_ast);
else
dest_children[exists - dest_children.begin()] = override_constraint_ast;
}
}
if (columns->projections)
{
for (const auto & override_projection_ast : columns->projections->children)
{
auto * override_projection = override_projection_ast->as<ASTProjectionDeclaration>();
if (!override_projection)
continue;
if (!create_query->columns_list->projections)
create_query->columns_list->set(create_query->columns_list->projections, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->projections->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTProjectionDeclaration>()->name == override_projection->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_projection_ast);
else
dest_children[exists - dest_children.begin()] = override_projection_ast;
}
}
}
if (storage)
{
if (!create_query->storage)
create_query->set(create_query->storage, std::make_shared<ASTStorage>());
if (storage->partition_by)
create_query->storage->set(create_query->storage->partition_by, storage->partition_by->clone());
if (storage->primary_key)
create_query->storage->set(create_query->storage->primary_key, storage->primary_key->clone());
if (storage->order_by)
create_query->storage->set(create_query->storage->order_by, storage->order_by->clone());
if (storage->sample_by)
create_query->storage->set(create_query->storage->sample_by, storage->sample_by->clone());
if (storage->ttl_table)
create_query->storage->set(create_query->storage->ttl_table, storage->ttl_table->clone());
// not supporting overriding ENGINE
}
}
ASTPtr ASTTableOverrideList::clone() const
{
auto res = std::make_shared<ASTTableOverrideList>(*this);
res->cloneChildren();
return res;
}
ASTPtr ASTTableOverrideList::tryGetTableOverride(const String & name) const
{
auto it = positions.find(name);
if (it == positions.end())
return nullptr;
return children[it->second];
}
void ASTTableOverrideList::setTableOverride(const String & name, const ASTPtr ast)
{
auto it = positions.find(name);
if (it == positions.end())
{
positions[name] = children.size();
children.emplace_back(ast);
}
else
{
children[it->second] = ast;
}
}
void ASTTableOverrideList::removeTableOverride(const String & name)
{
if (hasOverride(name))
{
size_t pos = positions[name];
children.erase(children.begin() + pos);
positions.erase(name);
for (auto & pr : positions)
if (pr.second > pos)
--pr.second;
}
}
bool ASTTableOverrideList::hasOverride(const String & name) const
{
return positions.count(name);
}
void ASTTableOverrideList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (frame.expression_list_prepend_whitespace)
settings.ostr << ' ';
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
{
settings.ostr << (settings.one_line ? ", " : ",\n");
}
(*it)->formatImpl(settings, state, frame);
}
}
}

View File

@ -0,0 +1,52 @@
#pragma once
#include <Parsers/IAST.h>
#include <map>
namespace DB
{
class ASTColumns;
class ASTCreateQuery;
class ASTIdentifier;
class ASTStorage;
/// Storage and column overrides for a single table, for example:
///
/// TABLE OVERRIDE `foo` PARTITION BY toYYYYMM(`createtime`)
///
class ASTTableOverride : public IAST
{
public:
String table_name;
ASTColumns * columns = nullptr;
ASTStorage * storage = nullptr;
String getID(char) const override { return "TableOverride " + table_name; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void applyToCreateTableQuery(ASTCreateQuery * create_query) const;
};
/// List of table overrides, for example:
///
/// TABLE OVERRIDE `foo` (PARTITION BY toYYYYMM(`createtime`)),
/// TABLE OVERRIDE `bar` (SAMPLE BY `id`)
///
class ASTTableOverrideList : public IAST
{
public:
String getID(char) const override { return "TableOverrideList"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void setTableOverride(const String & name, ASTPtr override);
void removeTableOverride(const String & name);
ASTPtr tryGetTableOverride(const String & name) const;
bool hasOverride(const String & name) const;
private:
std::map<String, size_t> positions;
};
}

View File

@ -1,22 +1,23 @@
#include <Common/typeid_cast.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTTableOverrides.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
#include <Parsers/ParserProjectionSelectQuery.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Common/typeid_cast.h>
namespace DB
@ -932,6 +933,141 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return true;
}
bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_table_override("TABLE OVERRIDE");
ParserIdentifier table_name_p;
ParserToken lparen_p(TokenType::OpeningRoundBracket);
ParserToken rparen_p(TokenType::ClosingRoundBracket);
ParserTablePropertiesDeclarationList table_properties_p;
ParserExpression expression_p;
ParserTTLExpressionList parser_ttl_list;
ParserKeyword s_columns("COLUMNS");
ParserKeyword s_partition_by("PARTITION BY");
ParserKeyword s_primary_key("PRIMARY KEY");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_ttl("TTL");
ASTPtr table_name;
ASTPtr columns;
ASTPtr partition_by;
ASTPtr primary_key;
ASTPtr order_by;
ASTPtr sample_by;
ASTPtr ttl_table;
if (!s_table_override.ignore(pos, expected))
return false;
if (!table_name_p.parse(pos, table_name, expected))
return false;
if (!lparen_p.ignore(pos, expected))
return false;
while (true)
{
if (!columns && s_columns.ignore(pos, expected))
{
if (!lparen_p.ignore(pos, expected))
return false;
if (!table_properties_p.parse(pos, columns, expected))
return false;
if (!rparen_p.ignore(pos, expected))
return false;
}
if (!partition_by && s_partition_by.ignore(pos, expected))
{
if (expression_p.parse(pos, partition_by, expected))
continue;
else
return false;
}
if (!primary_key && s_primary_key.ignore(pos, expected))
{
if (expression_p.parse(pos, primary_key, expected))
continue;
else
return false;
}
if (!order_by && s_order_by.ignore(pos, expected))
{
if (expression_p.parse(pos, order_by, expected))
continue;
else
return false;
}
if (!sample_by && s_sample_by.ignore(pos, expected))
{
if (expression_p.parse(pos, sample_by, expected))
continue;
else
return false;
}
if (!ttl_table && s_ttl.ignore(pos, expected))
{
if (parser_ttl_list.parse(pos, ttl_table, expected))
continue;
else
return false;
}
break;
}
if (!rparen_p.ignore(pos, expected))
return false;
auto storage = std::make_shared<ASTStorage>();
storage->set(storage->partition_by, partition_by);
storage->set(storage->primary_key, primary_key);
storage->set(storage->order_by, order_by);
storage->set(storage->sample_by, sample_by);
storage->set(storage->ttl_table, ttl_table);
auto res = std::make_shared<ASTTableOverride>();
res->table_name = table_name->as<ASTIdentifier>()->name();
res->set(res->storage, storage);
if (columns)
res->set(res->columns, columns);
node = res;
return true;
}
bool ParserTableOverridesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserTableOverrideDeclaration table_override_p;
ParserToken s_comma(TokenType::Comma);
auto res = std::make_shared<ASTTableOverrideList>();
auto parse_element = [&]
{
ASTPtr element;
if (!table_override_p.parse(pos, element, expected))
return false;
auto * table_override = element->as<ASTTableOverride>();
if (!table_override)
return false;
res->setTableOverride(table_override->table_name, element);
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_element, s_comma, true))
return false;
if (!res->children.empty())
node = res;
return true;
}
bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
@ -940,9 +1076,11 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserStorage storage_p;
ParserIdentifier name_p(true);
ParserTableOverridesDeclarationList table_overrides_p;
ASTPtr database;
ASTPtr storage;
ASTPtr table_overrides;
UUID uuid = UUIDHelpers::Nil;
String cluster_str;
@ -984,6 +1122,9 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
storage_p.parse(pos, storage, expected);
auto comment = parseComment(pos, expected);
if (!table_overrides_p.parse(pos, table_overrides, expected))
return false;
auto query = std::make_shared<ASTCreateQuery>();
node = query;
@ -1000,6 +1141,8 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
query->set(query->storage, storage);
if (comment)
query->set(query->comment, comment);
if (table_overrides && !table_overrides->children.empty())
query->set(query->table_overrides, table_overrides);
return true;
}

View File

@ -385,6 +385,20 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTableOverrideDeclaration : public IParserBase
{
protected:
const char * getName() const override { return "table override declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTableOverridesDeclarationList : public IParserBase
{
protected:
const char * getName() const override { return "table overrides declaration list"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// CREATE|ATTACH DATABASE db [ENGINE = engine]
class ParserCreateDatabaseQuery : public IParserBase
{

View File

@ -1,11 +1,13 @@
#include <Parsers/ParserOptimizeQuery.h>
#include <IO/WriteBufferFromOStream.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteBufferFromOStream.h>
#include <Parsers/parseQuery.h>
#include <string_view>
@ -25,7 +27,7 @@ struct ParserTestCase
std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr<IParser> parser)
{
return ostr << "Praser: " << parser->getName();
return ostr << "Parser: " << parser->getName();
}
std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case)
@ -55,6 +57,45 @@ TEST_P(ParserTest, parseQuery)
}
}
struct TableOverrideTestCase
{
DB::String create_database_query;
DB::String create_table_query;
DB::String expected_create_table_query;
};
std::ostream & operator<<(std::ostream & ostr, const TableOverrideTestCase & test_case)
{
return ostr << "database: " << test_case.create_database_query << ", table: " << test_case.create_table_query
<< ", expected: " << test_case.expected_create_table_query;
}
class TableOverrideTest : public ::testing::TestWithParam<TableOverrideTestCase>
{};
TEST_P(TableOverrideTest, applyOverrides)
{
const auto & [database_query, table_query, expected_query] = GetParam();
ParserCreateQuery parser;
ASTPtr database_ast;
ASSERT_NO_THROW(database_ast = parseQuery(parser, database_query, 0, 0));
auto * database = database_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, database);
ASTPtr table_ast;
ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0));
auto table = table_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, table);
auto table_name = table->table->as<ASTIdentifier>()->name();
if (database->table_overrides)
{
auto override_ast = database->table_overrides->tryGetTableOverride(table_name);
ASSERT_NE(nullptr, override_ast);
auto override = override_ast->as<ASTTableOverride>();
ASSERT_NE(nullptr, override);
override->applyToCreateTableQuery(table);
}
EXPECT_EQ(expected_query, serializeAST(*table));
}
INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery, ParserTest,
::testing::Combine(
@ -144,7 +185,7 @@ INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest,
INSTANTIATE_TEST_SUITE_P(ParserCreateQuery_DICTIONARY_WITH_COMMENT, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserAlterCommand>()),
::testing::Values(std::make_shared<ParserCreateQuery>()),
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
{
R"sql(CREATE DICTIONARY 2024_dictionary_with_comment
@ -170,3 +211,91 @@ LAYOUT(FLAT())
COMMENT 'Test dictionary with comment')sql"
}}
)));
INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserCreateQuery>()),
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
{
"CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')",
"CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')"
},
{
"CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))",
"CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n PARTITION BY toYYYYMM(`created`)\n)"
},
{
"CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())",
"CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE `tbl`,\nTABLE OVERRIDE `a`\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE `b`\n(\n PARTITION BY rand()\n)"
},
{
"CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))",
"CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(`created`)\n)"
},
{
"CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))",
"CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX foo `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)"
},
{
"CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)",
"CREATE DATABASE db\nTABLE OVERRIDE `t1`\n(\n TTL `inserted` + toIntervalMonth(1)\n),\nTABLE OVERRIDE `t2`\n(\n TTL `inserted` + toIntervalMonth(2)\n)"
},
{
"CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)",
"CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE `tab3`\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n),\nTABLE OVERRIDE `tab5`\n(\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n)"
},
{
"CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))",
"CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(`created`)\n)"
},
{
"CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1",
"CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1"
},
{
"CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2",
"CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2"
},
{
"CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))",
"CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)"
},
{
"CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))",
"CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)\nCOMMENT 'db comment'"
}
})));
INSTANTIATE_TEST_SUITE_P(ApplyTableOverrides, TableOverrideTest,
::testing::ValuesIn(std::initializer_list<TableOverrideTestCase>{
{
"CREATE DATABASE db",
"CREATE TABLE db.t (id Int64) ENGINE=Log",
"CREATE TABLE db.t (`id` Int64) ENGINE = Log"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (PARTITION BY tuple())",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` Int64) ENGINE = MergeTree PARTITION BY tuple()"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (COLUMNS (id UInt64, shard UInt8 MATERIALIZED modulo(id, 16)) PARTITION BY shard)",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` UInt64, `shard` UInt8 MATERIALIZED id % 16) ENGINE = MergeTree PARTITION BY shard"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.a (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.a (`id` Int64) ENGINE = MergeTree PARTITION BY id % 3"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.b (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.b (`id` Int64) ENGINE = MergeTree PARTITION BY id % 5"
},
{
"CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl` (PARTITION BY toYYYYMM(created))",
"CREATE TABLE db.tbl (id Int64, created DateTime) ENGINE=Foo",
"CREATE TABLE db.tbl (`id` Int64, `created` DateTime) ENGINE = Foo PARTITION BY toYYYYMM(created)",
}
}));

View File

@ -30,8 +30,21 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
bool with_types_,
bool ignore_spaces_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, buf, params_, with_names_, with_types_, updateFormatSettings(format_settings_))
, buf(in_)
: CustomSeparatedRowInputFormat(
header_, std::make_unique<PeekableReadBuffer>(in_), params_, with_names_, with_types_, ignore_spaces_, format_settings_)
{
}
CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
const Block & header_,
std::unique_ptr<PeekableReadBuffer> buf_,
const Params & params_,
bool with_names_,
bool with_types_,
bool ignore_spaces_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, *buf_, params_, with_names_, with_types_, updateFormatSettings(format_settings_))
, buf(std::move(buf_))
, ignore_spaces(ignore_spaces_)
, escaping_rule(format_settings_.custom.escaping_rule)
{
@ -51,57 +64,57 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
void CustomSeparatedRowInputFormat::skipPrefixBeforeHeader()
{
skipSpaces();
assertString(format_settings.custom.result_before_delimiter, buf);
assertString(format_settings.custom.result_before_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowStartDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_before_delimiter, buf);
assertString(format_settings.custom.row_before_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipFieldDelimiter()
{
skipSpaces();
assertString(format_settings.custom.field_delimiter, buf);
assertString(format_settings.custom.field_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowEndDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_after_delimiter, buf);
assertString(format_settings.custom.row_after_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowBetweenDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_between_delimiter, buf);
assertString(format_settings.custom.row_between_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipField()
{
skipSpaces();
skipFieldByEscapingRule(buf, escaping_rule, format_settings);
skipFieldByEscapingRule(*buf, escaping_rule, format_settings);
}
bool CustomSeparatedRowInputFormat::checkEndOfRow()
{
PeekableReadBufferCheckpoint checkpoint{buf, true};
PeekableReadBufferCheckpoint checkpoint{*buf, true};
skipSpaces();
if (!checkString(format_settings.custom.row_after_delimiter, buf))
if (!checkString(format_settings.custom.row_after_delimiter, *buf))
return false;
skipSpaces();
/// At the end of row after row_after_delimiter we expect result_after_delimiter or row_between_delimiter.
if (checkString(format_settings.custom.row_between_delimiter, buf))
if (checkString(format_settings.custom.row_between_delimiter, *buf))
return true;
buf.rollbackToCheckpoint();
buf->rollbackToCheckpoint();
skipSpaces();
buf.ignore(format_settings.custom.row_after_delimiter.size());
buf->ignore(format_settings.custom.row_after_delimiter.size());
return checkForSuffixImpl(true);
}
@ -114,7 +127,7 @@ std::vector<String> CustomSeparatedRowInputFormat::readHeaderRow()
if (!values.empty())
skipFieldDelimiter();
skipSpaces();
values.push_back(readStringByEscapingRule(buf, escaping_rule, format_settings));
values.push_back(readStringByEscapingRule(*buf, escaping_rule, format_settings));
}
while (!checkEndOfRow());
@ -138,7 +151,7 @@ void CustomSeparatedRowInputFormat::skipHeaderRow()
bool CustomSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &)
{
skipSpaces();
return deserializeFieldByEscapingRule(type, serialization, column, buf, escaping_rule, format_settings);
return deserializeFieldByEscapingRule(type, serialization, column, *buf, escaping_rule, format_settings);
}
bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof)
@ -149,16 +162,16 @@ bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof)
if (!check_eof)
return false;
return buf.eof();
return buf->eof();
}
if (unlikely(checkString(format_settings.custom.result_after_delimiter, buf)))
if (unlikely(checkString(format_settings.custom.result_after_delimiter, *buf)))
{
skipSpaces();
if (!check_eof)
return true;
if (buf.eof())
if (buf->eof())
return true;
}
return false;
@ -166,25 +179,25 @@ bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof)
bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out)
{
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
if (checkForSuffixImpl(false))
{
if (buf.eof())
if (buf->eof())
out << "<End of stream>\n";
else
out << " There is some data after suffix\n";
return false;
}
buf.rollbackToCheckpoint();
buf->rollbackToCheckpoint();
return true;
}
bool CustomSeparatedRowInputFormat::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
if (checkForSuffixImpl(true))
return true;
buf.rollbackToCheckpoint();
buf->rollbackToCheckpoint();
return false;
}
@ -196,37 +209,43 @@ bool CustomSeparatedRowInputFormat::allowSyncAfterError() const
void CustomSeparatedRowInputFormat::syncAfterError()
{
skipToNextRowOrEof(buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces);
end_of_stream = buf.eof();
/// It can happen that buf.position() is not at the beginning of row
skipToNextRowOrEof(*buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces);
end_of_stream = buf->eof();
/// It can happen that buf->position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces);
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.field_delimiter, "delimiter between fields", ignore_spaces);
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.field_delimiter, "delimiter between fields", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_after_delimiter, "delimiter after last field", ignore_spaces);
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_after_delimiter, "delimiter after last field", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_between_delimiter, "delimiter between rows", ignore_spaces);
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_between_delimiter, "delimiter between rows", ignore_spaces);
}
void CustomSeparatedRowInputFormat::resetParser()
{
RowInputFormatWithNamesAndTypes::resetParser();
buf.reset();
buf->reset();
}
void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatCustomSeparated(FormatFactory & factory)

View File

@ -20,6 +20,8 @@ public:
void resetParser() override;
String getName() const override { return "CustomSeparatedRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
private:
CustomSeparatedRowInputFormat(
const Block & header_,
@ -59,9 +61,9 @@ private:
bool checkEndOfRow();
bool checkForSuffixImpl(bool check_eof);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); }
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
PeekableReadBuffer buf;
std::unique_ptr<PeekableReadBuffer> buf;
bool ignore_spaces;
EscapingRule escaping_rule;
};

View File

@ -14,8 +14,11 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
}
JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) :
IRowInputFormat(header_, buf, std::move(params_)), buf(in_)
JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
: JSONAsStringRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_) {}
JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_) :
IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_))
{
if (header_.columns() > 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
@ -31,113 +34,113 @@ JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, Re
void JSONAsStringRowInputFormat::resetParser()
{
IRowInputFormat::resetParser();
buf.reset();
buf->reset();
}
void JSONAsStringRowInputFormat::readPrefix()
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(buf);
skipBOMIfExists(*buf);
skipWhitespaceIfAny(buf);
if (!buf.eof() && *buf.position() == '[')
skipWhitespaceIfAny(*buf);
if (!buf->eof() && *buf->position() == '[')
{
++buf.position();
++buf->position();
data_in_square_brackets = true;
}
}
void JSONAsStringRowInputFormat::readSuffix()
{
skipWhitespaceIfAny(buf);
skipWhitespaceIfAny(*buf);
if (data_in_square_brackets)
{
assertChar(']', buf);
skipWhitespaceIfAny(buf);
assertChar(']', *buf);
skipWhitespaceIfAny(*buf);
}
if (!buf.eof() && *buf.position() == ';')
if (!buf->eof() && *buf->position() == ';')
{
++buf.position();
skipWhitespaceIfAny(buf);
++buf->position();
skipWhitespaceIfAny(*buf);
}
assertEOF(buf);
assertEOF(*buf);
}
void JSONAsStringRowInputFormat::readJSONObject(IColumn & column)
{
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
size_t balance = 0;
bool quotes = false;
if (*buf.position() != '{')
if (*buf->position() != '{')
throw Exception("JSON object must begin with '{'.", ErrorCodes::INCORRECT_DATA);
++buf.position();
++buf->position();
++balance;
char * pos;
while (balance)
{
if (buf.eof())
if (buf->eof())
throw Exception("Unexpected end of file while parsing JSON object.", ErrorCodes::INCORRECT_DATA);
if (quotes)
{
pos = find_first_symbols<'"', '\\'>(buf.position(), buf.buffer().end());
buf.position() = pos;
if (buf.position() == buf.buffer().end())
pos = find_first_symbols<'"', '\\'>(buf->position(), buf->buffer().end());
buf->position() = pos;
if (buf->position() == buf->buffer().end())
continue;
if (*buf.position() == '"')
if (*buf->position() == '"')
{
quotes = false;
++buf.position();
++buf->position();
}
else if (*buf.position() == '\\')
else if (*buf->position() == '\\')
{
++buf.position();
if (!buf.eof())
++buf->position();
if (!buf->eof())
{
++buf.position();
++buf->position();
}
}
}
else
{
pos = find_first_symbols<'"', '{', '}', '\\'>(buf.position(), buf.buffer().end());
buf.position() = pos;
if (buf.position() == buf.buffer().end())
pos = find_first_symbols<'"', '{', '}', '\\'>(buf->position(), buf->buffer().end());
buf->position() = pos;
if (buf->position() == buf->buffer().end())
continue;
if (*buf.position() == '{')
if (*buf->position() == '{')
{
++balance;
++buf.position();
++buf->position();
}
else if (*buf.position() == '}')
else if (*buf->position() == '}')
{
--balance;
++buf.position();
++buf->position();
}
else if (*buf.position() == '\\')
else if (*buf->position() == '\\')
{
++buf.position();
if (!buf.eof())
++buf->position();
if (!buf->eof())
{
++buf.position();
++buf->position();
}
}
else if (*buf.position() == '"')
else if (*buf->position() == '"')
{
quotes = true;
++buf.position();
++buf->position();
}
}
}
buf.makeContinuousMemoryFromCheckpointToPos();
char * end = buf.position();
buf.rollbackToCheckpoint();
column.insertData(buf.position(), end - buf.position());
buf.position() = end;
buf->makeContinuousMemoryFromCheckpointToPos();
char * end = buf->position();
buf->rollbackToCheckpoint();
column.insertData(buf->position(), end - buf->position());
buf->position() = end;
}
bool JSONAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
@ -145,30 +148,36 @@ bool JSONAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
if (!allow_new_rows)
return false;
skipWhitespaceIfAny(buf);
if (!buf.eof())
skipWhitespaceIfAny(*buf);
if (!buf->eof())
{
if (!data_in_square_brackets && *buf.position() == ';')
if (!data_in_square_brackets && *buf->position() == ';')
{
/// ';' means the end of query, but it cannot be before ']'.
return allow_new_rows = false;
}
else if (data_in_square_brackets && *buf.position() == ']')
else if (data_in_square_brackets && *buf->position() == ']')
{
/// ']' means the end of query.
return allow_new_rows = false;
}
}
if (!buf.eof())
if (!buf->eof())
readJSONObject(*columns[0]);
skipWhitespaceIfAny(buf);
if (!buf.eof() && *buf.position() == ',')
++buf.position();
skipWhitespaceIfAny(buf);
skipWhitespaceIfAny(*buf);
if (!buf->eof() && *buf->position() == ',')
++buf->position();
skipWhitespaceIfAny(*buf);
return !buf.eof();
return !buf->eof();
}
void JSONAsStringRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatJSONAsString(FormatFactory & factory)

View File

@ -20,8 +20,11 @@ public:
String getName() const override { return "JSONAsStringRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
private:
JSONAsStringRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_);
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
@ -29,7 +32,7 @@ private:
void readJSONObject(IColumn & column);
PeekableReadBuffer buf;
std::unique_ptr<PeekableReadBuffer> buf;
/// This flag is needed to know if data is in square brackets.
bool data_in_square_brackets = false;

View File

@ -29,12 +29,15 @@ namespace ErrorCodes
}
MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
: IRowInputFormat(header_, buf, std::move(params_)), buf(in_), parser(visitor), data_types(header_.getDataTypes()) {}
: MsgPackRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_) {}
MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_)
: IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_)), parser(visitor), data_types(header_.getDataTypes()) {}
void MsgPackRowInputFormat::resetParser()
{
IRowInputFormat::resetParser();
buf.reset();
buf->reset();
visitor.reset();
}
@ -325,21 +328,21 @@ void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT
bool MsgPackRowInputFormat::readObject()
{
if (buf.eof())
if (buf->eof())
return false;
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
size_t offset = 0;
while (!parser.execute(buf.position(), buf.available(), offset))
while (!parser.execute(buf->position(), buf->available(), offset))
{
buf.position() = buf.buffer().end();
if (buf.eof())
buf->position() = buf->buffer().end();
if (buf->eof())
throw Exception("Unexpected end of file while parsing msgpack object.", ErrorCodes::INCORRECT_DATA);
buf.position() = buf.buffer().end();
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
buf->position() = buf->buffer().end();
buf->makeContinuousMemoryFromCheckpointToPos();
buf->rollbackToCheckpoint();
}
buf.position() += offset;
buf->position() += offset;
return true;
}
@ -363,6 +366,12 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
return true;
}
void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatMsgPack(FormatFactory & factory)
{
factory.registerInputFormat("MsgPack", [](

View File

@ -61,13 +61,16 @@ public:
String getName() const override { return "MagPackRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
private:
MsgPackRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_);
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool readObject();
PeekableReadBuffer buf;
std::unique_ptr<PeekableReadBuffer> buf;
MsgPackVisitor visitor;
msgpack::detail::parse_helper<MsgPackVisitor> parser;
const DataTypes data_types;

View File

@ -325,7 +325,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding(
{
String serialized_value = " ";
{
WriteBufferFromString out_serialize(serialized_value, WriteBufferFromString::AppendModeTag());
WriteBufferFromString out_serialize(serialized_value, AppendModeTag());
serialization.serializeText(column, row_num, out_serialize, format_settings);
}

View File

@ -14,10 +14,15 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
RegexpRowInputFormat::RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: RegexpRowInputFormat(std::make_unique<PeekableReadBuffer>(in_), header_, params_, format_settings_)
{
}
RegexpRowInputFormat::RegexpRowInputFormat(
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, buf, std::move(params_))
, buf(in_)
std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, *buf_, std::move(params_))
, buf(std::move(buf_))
, format_settings(format_settings_)
, escaping_rule(format_settings_.regexp.escaping_rule)
, regexp(format_settings_.regexp.regexp)
@ -39,7 +44,7 @@ RegexpRowInputFormat::RegexpRowInputFormat(
void RegexpRowInputFormat::resetParser()
{
IRowInputFormat::resetParser();
buf.reset();
buf->reset();
}
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
@ -71,45 +76,51 @@ void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowRead
bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
{
if (buf.eof())
if (buf->eof())
return false;
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
size_t line_size = 0;
do
{
char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end());
line_size += pos - buf.position();
buf.position() = pos;
} while (buf.position() == buf.buffer().end() && !buf.eof());
char * pos = find_first_symbols<'\n', '\r'>(buf->position(), buf->buffer().end());
line_size += pos - buf->position();
buf->position() = pos;
} while (buf->position() == buf->buffer().end() && !buf->eof());
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
buf->makeContinuousMemoryFromCheckpointToPos();
buf->rollbackToCheckpoint();
bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
bool match = RE2::FullMatchN(re2::StringPiece(buf->position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
bool read_line = true;
if (!match)
{
if (!format_settings.regexp.skip_unmatched)
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
throw Exception("Line \"" + std::string(buf->position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
read_line = false;
}
if (read_line)
readFieldsFromMatch(columns, ext);
buf.position() += line_size;
buf->position() += line_size;
checkChar('\r', buf);
if (!buf.eof() && !checkChar('\n', buf))
checkChar('\r', *buf);
if (!buf->eof() && !checkChar('\n', *buf))
throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA);
return true;
}
void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatRegexp(FormatFactory & factory)
{
factory.registerInputFormat("Regexp", [](

View File

@ -31,14 +31,17 @@ public:
String getName() const override { return "RegexpRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
private:
RegexpRowInputFormat(std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_);
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool readField(size_t index, MutableColumns & columns);
void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext);
PeekableReadBuffer buf;
std::unique_ptr<PeekableReadBuffer> buf;
const FormatSettings format_settings;
const EscapingRule escaping_rule;

View File

@ -20,11 +20,25 @@ extern const int SYNTAX_ERROR;
}
TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
TemplateRowInputFormat::TemplateRowInputFormat(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
FormatSettings settings_,
bool ignore_spaces_,
ParsedTemplateFormatString format_,
ParsedTemplateFormatString row_format_,
std::string row_between_delimiter_)
: TemplateRowInputFormat(
header_, std::make_unique<PeekableReadBuffer>(in_), params_, settings_, ignore_spaces_, format_, row_format_, row_between_delimiter_)
{
}
TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, const Params & params_,
FormatSettings settings_, bool ignore_spaces_,
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_,
std::string row_between_delimiter_)
: RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()),
: RowInputFormatWithDiagnosticInfo(header_, *buf_, params_), buf(std::move(buf_)), data_types(header_.getDataTypes()),
settings(std::move(settings_)), ignore_spaces(ignore_spaces_),
format(std::move(format_)), row_format(std::move(row_format_)),
default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(std::move(row_between_delimiter_))
@ -101,10 +115,10 @@ ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
@ -133,10 +147,10 @@ ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
}
@ -162,14 +176,14 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
updateDiagnosticInfo();
if (likely(row_num != 1))
assertString(row_between_delimiter, buf);
assertString(row_between_delimiter, *buf);
extra.read_columns.assign(columns.size(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
assertString(row_format.delimiters[i], buf);
assertString(row_format.delimiters[i], *buf);
skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
@ -182,7 +196,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
}
skipSpaces();
assertString(row_format.delimiters.back(), buf);
assertString(row_format.delimiters.back(), *buf);
for (const auto & idx : always_default_columns)
data_types[idx]->insertDefaultInto(*columns[idx]);
@ -200,7 +214,7 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
row_format.delimiters[file_column + 1].front();
try
{
return deserializeFieldByEscapingRule(type, serialization, column, buf, escaping_rule, settings);
return deserializeFieldByEscapingRule(type, serialization, column, *buf, escaping_rule, settings);
}
catch (Exception & e)
{
@ -214,7 +228,7 @@ void TemplateRowInputFormat::skipField(TemplateRowInputFormat::EscapingRule esca
{
try
{
skipFieldByEscapingRule(buf, escaping_rule, settings);
skipFieldByEscapingRule(*buf, escaping_rule, settings);
}
catch (Exception & e)
{
@ -228,7 +242,7 @@ void TemplateRowInputFormat::skipField(TemplateRowInputFormat::EscapingRule esca
/// Otherwise returns false
bool TemplateRowInputFormat::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{buf};
PeekableReadBufferCheckpoint checkpoint{*buf};
bool suffix_found = false;
size_t last_successfully_parsed_idx = format_data_idx + 1;
try
@ -246,11 +260,11 @@ bool TemplateRowInputFormat::checkForSuffix()
if (unlikely(suffix_found))
{
skipSpaces();
if (buf.eof())
if (buf->eof())
return true;
}
buf.rollbackToCheckpoint();
buf->rollbackToCheckpoint();
return false;
}
@ -258,11 +272,11 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
{
out << "Suffix does not match: ";
size_t last_successfully_parsed_idx = format_data_idx + 1;
const ReadBuffer::Position row_begin_pos = buf.position();
const ReadBuffer::Position row_begin_pos = buf->position();
bool caught = false;
try
{
PeekableReadBufferCheckpoint checkpoint{buf, true};
PeekableReadBufferCheckpoint checkpoint{*buf, true};
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format.columnsCount());
}
catch (Exception & e)
@ -273,12 +287,12 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
if (!caught)
{
out << " There is some data after suffix (EOF expected, got ";
verbosePrintString(buf.position(), std::min(buf.buffer().end(), buf.position() + 16), out);
verbosePrintString(buf->position(), std::min(buf->buffer().end(), buf->position() + 16), out);
out << "). ";
}
out << " Format string (from format_schema): \n" << format.dump() << "\n";
if (row_begin_pos != buf.position())
if (row_begin_pos != buf->position())
{
/// Pointers to buffer memory were invalidated during checking for suffix
out << "\nCannot print more diagnostic info.";
@ -287,12 +301,12 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n";
out << "\nTrying to parse next row, because suffix does not match:\n";
if (likely(row_num != 1) && !parseDelimiterWithDiagnosticInfo(out, buf, row_between_delimiter, "delimiter between rows", ignore_spaces))
if (likely(row_num != 1) && !parseDelimiterWithDiagnosticInfo(out, *buf, row_between_delimiter, "delimiter between rows", ignore_spaces))
return false;
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (!parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters[i], "delimiter before field " + std::to_string(i), ignore_spaces))
if (!parseDelimiterWithDiagnosticInfo(out, *buf, row_format.delimiters[i], "delimiter before field " + std::to_string(i), ignore_spaces))
return false;
skipSpaces();
@ -318,7 +332,7 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
}
}
return parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters.back(), "delimiter after last field", ignore_spaces);
return parseDelimiterWithDiagnosticInfo(out, *buf, row_format.delimiters.back(), "delimiter after last field", ignore_spaces);
}
bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces)
@ -366,9 +380,9 @@ bool TemplateRowInputFormat::allowSyncAfterError() const
void TemplateRowInputFormat::syncAfterError()
{
skipToNextRowOrEof(buf, row_format.delimiters.back(), row_between_delimiter, ignore_spaces);
end_of_stream = buf.eof();
/// It can happen that buf.position() is not at the beginning of row
skipToNextRowOrEof(*buf, row_format.delimiters.back(), row_between_delimiter, ignore_spaces);
end_of_stream = buf->eof();
/// It can happen that buf->position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
@ -384,7 +398,13 @@ void TemplateRowInputFormat::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
end_of_stream = false;
buf.reset();
buf->reset();
}
void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatTemplate(FormatFactory & factory)

View File

@ -25,6 +25,11 @@ public:
void resetParser() override;
private:
TemplateRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, const Params & params_,
FormatSettings settings_, bool ignore_spaces_,
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_,
std::string row_between_delimiter);
bool readRow(MutableColumns & columns, RowReadExtension & extra) override;
void readPrefix() override;
@ -36,7 +41,7 @@ private:
const SerializationPtr & serialization, IColumn & column, size_t file_column);
void skipField(EscapingRule escaping_rule);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); }
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
template <typename ReturnType = void>
ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end);
@ -48,7 +53,9 @@ private:
bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override;
PeekableReadBuffer buf;
void setReadBuffer(ReadBuffer & in_) override;
std::unique_ptr<PeekableReadBuffer> buf;
const DataTypes data_types;
FormatSettings settings;

View File

@ -61,8 +61,12 @@ static String formattedAST(const ASTPtr & ast)
{
if (!ast)
return {};
WriteBufferFromOwnString buf;
formatAST(*ast, buf, false, true);
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true);
ast_format_settings.hilite = false;
ast_format_settings.always_quote_identifiers = true;
ast->format(ast_format_settings);
return buf.str();
}

View File

@ -31,13 +31,13 @@ struct BlockIO
/// When it is true, don't bother sending any non-empty blocks to the out stream
bool null_format = false;
/// Call these functions if you want to log the request.
void onFinish()
{
if (finish_callback)
{
finish_callback(pipeline);
}
pipeline.reset();
}
void onException() const

View File

@ -48,6 +48,7 @@ void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String p
}
else
{
std::sort(potential_leaders.begin(), potential_leaders.end());
if (potential_leaders.front() == persistent_multiple_leaders)
return;

View File

@ -36,7 +36,7 @@
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/IAST.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
@ -730,7 +730,15 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer
std::vector<std::unique_ptr<QueryPipelineBuilder>> pipelines;
String new_query_str = queryToString(new_query);
String new_query_str;
{
WriteBufferFromOwnString buf;
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true);
ast_format_settings.always_quote_identifiers = true;
new_query->IAST::format(ast_format_settings);
new_query_str = buf.str();
}
for (size_t shard_index : collections::range(0, shards_info.size()))
{
const auto & shard_info = shards_info[shard_index];

View File

@ -147,8 +147,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user
Strings paths;
/// Do not use fs::canonical or fs::weakly_canonical.
/// Otherwise it will not allow to work with symlinks in `user_files_path` directory.
String path = fs::absolute(fs_table_path);
path = fs::path(path).lexically_normal(); /// Normalize path.
String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path.
if (path.find_first_of("*?{") == std::string::npos)
{
std::error_code error;

View File

@ -216,6 +216,13 @@ ColumnPtr fillColumnWithRandomData(
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt16), rng);
return column;
}
case TypeIndex::Date32:
{
auto column = ColumnInt32::create();
column->getData().resize(limit);
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Int32), rng);
return column;
}
case TypeIndex::UInt32: [[fallthrough]];
case TypeIndex::DateTime:
{

View File

@ -254,16 +254,24 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet
for (const auto & projection : getProjections())
add_dependent_columns(&projection, projections_columns);
if (hasRowsTTL())
auto add_for_rows_ttl = [&](const auto & expression, auto & to_set)
{
auto rows_expression = getRowsTTL().expression;
if (add_dependent_columns(rows_expression, required_ttl_columns) && include_ttl_target)
if (add_dependent_columns(expression, to_set) && include_ttl_target)
{
/// Filter all columns, if rows TTL expression have to be recalculated.
for (const auto & column : getColumns().getAllPhysical())
updated_ttl_columns.insert(column.name);
}
}
};
if (hasRowsTTL())
add_for_rows_ttl(getRowsTTL().expression, required_ttl_columns);
for (const auto & entry : getRowsWhereTTLs())
add_for_rows_ttl(entry.expression, required_ttl_columns);
for (const auto & entry : getGroupByTTLs())
add_for_rows_ttl(entry.expression, required_ttl_columns);
for (const auto & entry : getRecompressionTTLs())
add_dependent_columns(entry.expression, required_ttl_columns);

View File

@ -66,10 +66,17 @@ void StorageMongoDB::connectIfNotConnected()
if (!authenticated)
{
Poco::URI poco_uri(uri);
auto query_params = poco_uri.getQueryParameters();
auto auth_source = std::find_if(query_params.begin(), query_params.end(),
[&](const std::pair<std::string, std::string> & param) { return param.first == "authSource"; });
auto auth_db = database_name;
if (auth_source != query_params.end())
auth_db = auth_source->second;
# if POCO_VERSION >= 0x01070800
if (!username.empty() && !password.empty())
{
Poco::MongoDB::Database poco_db(database_name);
Poco::MongoDB::Database poco_db(auth_db);
if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1))
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}

View File

@ -74,7 +74,7 @@ class Backport:
# First pass. Find all must-backports
for label in pr['labels']['nodes']:
if label['name'] == 'pr-bugfix' or label['name'] == 'pr-must-backport':
if label['name'] == 'pr-must-backport':
backport_map[pr['number']] = branch_set.copy()
continue
matched = RE_MUST_BACKPORT.match(label['name'])

View File

@ -21,7 +21,7 @@ DO_NOT_TEST_LABEL = "do not test"
# Individual trusted contirbutors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
# a trusted org, to save GitHub API calls.
TRUSTED_CONTRIBUTORS = {
TRUSTED_CONTRIBUTORS = {e.lower() for e in [
"achimbab",
"adevyatova ", # DOCSUP
"Algunenano", # Raúl Marín, Tinybird
@ -64,12 +64,12 @@ TRUSTED_CONTRIBUTORS = {
"vdimir", # Employee
"vzakaznikov",
"YiuRULE",
"zlobober" # Developer of YT
}
"zlobober", # Developer of YT
]}
def pr_is_by_trusted_user(pr_user_login, pr_user_orgs):
if pr_user_login in TRUSTED_CONTRIBUTORS:
if pr_user_login.lower() in TRUSTED_CONTRIBUTORS:
logging.info("User '%s' is trusted", pr_user_login)
return True

View File

@ -43,12 +43,13 @@ NEED_RERUN_WORKFLOWS = {
13241696, # PR
15834118, # Docs
15522500, # MasterCI
15516108, # ReleaseCI
}
# Individual trusted contirbutors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
# a trusted org, to save GitHub API calls.
TRUSTED_CONTRIBUTORS = {
TRUSTED_CONTRIBUTORS = {e.lower() for e in [
"achimbab",
"adevyatova ", # DOCSUP
"Algunenano", # Raúl Marín, Tinybird
@ -88,8 +89,8 @@ TRUSTED_CONTRIBUTORS = {
"vdimir", # Employee
"vzakaznikov",
"YiuRULE",
"zlobober" # Developer of YT
}
"zlobober", # Developer of YT
]}
def get_installation_id(jwt_token):
@ -126,7 +127,7 @@ def get_key_and_app_from_aws():
def is_trusted_contributor(pr_user_login, pr_user_orgs):
if pr_user_login in TRUSTED_CONTRIBUTORS:
if pr_user_login.lower() in TRUSTED_CONTRIBUTORS:
print(f"User '{pr_user_login}' is trusted")
return True

View File

@ -285,6 +285,8 @@ class ClickHouseCluster:
self.minio_redirect_ip = None
self.minio_redirect_port = 8080
self.with_azurite = False
# available when with_hdfs == True
self.hdfs_host = "hdfs1"
self.hdfs_ip = None
@ -744,6 +746,13 @@ class ClickHouseCluster:
'--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]
return self.base_minio_cmd
def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_azurite = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_azurite.yml')])
self.base_azurite_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name,
'--file', p.join(docker_compose_yml_dir, 'docker_compose_azurite.yml')]
return self.base_azurite_cmd
def setup_cassandra_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_cassandra = True
env_variables['CASSANDRA_PORT'] = str(self.cassandra_port)
@ -775,7 +784,7 @@ class ClickHouseCluster:
with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None,
with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False,
with_kerberized_hdfs=False, with_mongo=False, with_mongo_secure=False, with_nginx=False,
with_redis=False, with_minio=False, with_cassandra=False, with_jdbc_bridge=False,
with_redis=False, with_minio=False, with_azurite=False, with_cassandra=False, with_jdbc_bridge=False,
hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None,
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None,
zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True,
@ -831,6 +840,7 @@ class ClickHouseCluster:
with_mongo=with_mongo or with_mongo_secure,
with_redis=with_redis,
with_minio=with_minio,
with_azurite=with_azurite,
with_cassandra=with_cassandra,
with_jdbc_bridge=with_jdbc_bridge,
server_bin_path=self.server_bin_path,
@ -934,6 +944,9 @@ class ClickHouseCluster:
if with_minio and not self.with_minio:
cmds.append(self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir))
if with_azurite and not self.with_azurite:
cmds.append(self.setup_azurite_cmd(instance, env_variables, docker_compose_yml_dir))
if minio_certs_dir is not None:
if self.minio_certs_dir is None:
self.minio_certs_dir = minio_certs_dir
@ -1385,6 +1398,23 @@ class ClickHouseCluster:
raise Exception("Can't wait Minio to start")
def wait_azurite_to_start(self, timeout=180):
from azure.storage.blob import BlobServiceClient
connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;"
time.sleep(1)
start = time.time()
while time.time() - start < timeout:
try:
blob_service_client = BlobServiceClient.from_connection_string(connection_string)
logging.debug(blob_service_client.get_account_information())
self.blob_service_client = blob_service_client
return
except Exception as ex:
logging.debug("Can't connect to Azurite: %s", str(ex))
time.sleep(1)
raise Exception("Can't wait Azurite to start")
def wait_schema_registry_to_start(self, timeout=180):
sr_client = CachedSchemaRegistryClient({"url":'http://localhost:{}'.format(self.schema_registry_port)})
start = time.time()
@ -1626,6 +1656,14 @@ class ClickHouseCluster:
logging.info("Trying to connect to Minio...")
self.wait_minio_to_start(secure=self.minio_certs_dir is not None)
if self.with_azurite and self.base_azurite_cmd:
azurite_start_cmd = self.base_azurite_cmd + common_opts
logging.info("Trying to create Azurite instance by command %s", ' '.join(map(str, azurite_start_cmd)))
run_and_check(azurite_start_cmd)
self.up_called = True
logging.info("Trying to connect to Azurite")
self.wait_azurite_to_start()
if self.with_cassandra and self.base_cassandra_cmd:
subprocess_check_call(self.base_cassandra_cmd + ['up', '-d'])
self.up_called = True
@ -1845,7 +1883,7 @@ class ClickHouseInstance:
self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs,
custom_dictionaries,
macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka,
with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge,
with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_azurite, with_jdbc_bridge,
with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True,
@ -1889,6 +1927,7 @@ class ClickHouseInstance:
self.with_mongo = with_mongo
self.with_redis = with_redis
self.with_minio = with_minio
self.with_azurite = with_azurite
self.with_cassandra = with_cassandra
self.with_jdbc_bridge = with_jdbc_bridge
@ -2117,7 +2156,7 @@ class ClickHouseInstance:
def wait_start(self, start_wait_sec):
start_time = time.time()
last_err = None
while time.time() <= start_time + start_wait_sec:
while True:
try:
pid = self.get_process_pid("clickhouse")
if pid is None:
@ -2131,6 +2170,8 @@ class ClickHouseInstance:
logging.warning(f"ERROR {err}")
else:
raise Exception("ClickHouse server is not running. Check logs.")
if time.time() > start_time + start_wait_sec:
break
logging.error(f"No time left to start. But process is still running. Will dump threads.")
ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], nothrow=True, user='root')
logging.info(f"PS RESULT:\n{ps_clickhouse}")
@ -2552,6 +2593,9 @@ class ClickHouseInstance:
if self.with_minio:
depends_on.append("minio1")
if self.with_azurite:
depends_on.append("azurite1")
self.cluster.env_variables.update(self.env_variables)
odbc_ini_path = ""

View File

@ -0,0 +1,43 @@
import string
import random
import threading
# By default the exceptions that was throwed in threads will be ignored
# (they will not mark the test as failed, only printed to stderr).
# Wrap thrading.Thread and re-throw exception on join()
class SafeThread(threading.Thread):
def __init__(self, target):
super().__init__()
self.target = target
self.exception = None
def run(self):
try:
self.target()
except Exception as e: # pylint: disable=broad-except
self.exception = e
def join(self, timeout=None):
super().join(timeout)
if self.exception:
raise self.exception
def random_string(length):
letters = string.ascii_letters
return ''.join(random.choice(letters) for i in range(length))
def generate_values(date_str, count, sign=1):
data = [[date_str, sign * (i + 1), random_string(10)] for i in range(count)]
data.sort(key=lambda tup: tup[1])
return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data])
def replace_config(config_path, old, new):
config = open(config_path, 'r')
config_lines = config.readlines()
config.close()
config_lines = [line.replace(old, new) for line in config_lines]
config = open(config_path, 'w')
config.writelines(config_lines)
config.close()

View File

@ -15,3 +15,7 @@ log_file = pytest.log
log_file_level = DEBUG
log_file_format = %(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)
log_file_date_format = %Y-%m-%d %H:%M:%S
markers =
long_run: marks tests which run for a long time
addopts =
-m 'not long_run'

View File

@ -0,0 +1,46 @@
<clickhouse>
<storage_configuration>
<disks>
<blob_storage_disk>
<type>blob_storage</type>
<storage_account_url>http://azurite1:10000/devstoreaccount1</storage_account_url>
<container_name>cont</container_name>
<skip_access_check>false</skip_access_check>
<!-- default credentials for Azurite storage account -->
<account_name>devstoreaccount1</account_name>
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
</blob_storage_disk>
</disks>
<policies>
<blob_storage_policy>
<volumes>
<main>
<disk>blob_storage_disk</disk>
</main>
</volumes>
</blob_storage_policy>
</policies>
</storage_configuration>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
<macros>
<cluster>test_cluster</cluster>
</macros>
</clickhouse>

View File

@ -0,0 +1,83 @@
import logging
import pytest
from helpers.cluster import ClickHouseCluster
logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler())
NODE1 = "node1"
NODE2 = "node2"
TABLE_NAME = "blob_storage_table"
CONTAINER_NAME = "cont"
CLUSTER_NAME = "test_cluster"
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(NODE1, main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '1'},
with_azurite=True,
with_zookeeper=True)
cluster.add_instance(NODE2, main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '2'},
with_azurite=True,
with_zookeeper=True)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def create_table(node, table_name, replica, **additional_settings):
settings = {
"storage_policy": "blob_storage_policy",
"old_parts_lifetime": 1,
}
settings.update(additional_settings)
create_table_statement = f"""
CREATE TABLE {table_name} ON CLUSTER {CLUSTER_NAME} (
id Int64,
data String
) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{table_name}', '{{replica}}')
ORDER BY id
SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}"""
node.query(f"DROP TABLE IF EXISTS {table_name}")
node.query(create_table_statement)
assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)"
def get_large_objects_count(blob_container_client, large_size_threshold=100):
return sum(blob['size'] > large_size_threshold for blob in blob_container_client.list_blobs())
def test_zero_copy_replication(cluster):
node1 = cluster.instances[NODE1]
node2 = cluster.instances[NODE2]
create_table(node1, TABLE_NAME, 1)
blob_container_client = cluster.blob_service_client.get_container_client(CONTAINER_NAME)
values1 = "(0,'data'),(1,'data')"
values2 = "(2,'data'),(3,'data')"
node1.query(f"INSERT INTO {TABLE_NAME} VALUES {values1}")
node2.query(f"SYSTEM SYNC REPLICA {TABLE_NAME}")
assert node1.query(f"SELECT * FROM {TABLE_NAME} order by id FORMAT Values") == values1
assert node2.query(f"SELECT * FROM {TABLE_NAME} order by id FORMAT Values") == values1
# Based on version 21.x - should be only one file with size 100+ (checksums.txt), used by both nodes
assert get_large_objects_count(blob_container_client) == 1
node2.query(f"INSERT INTO {TABLE_NAME} VALUES {values2}")
node1.query(f"SYSTEM SYNC REPLICA {TABLE_NAME}")
assert node2.query(f"SELECT * FROM {TABLE_NAME} order by id FORMAT Values") == values1 + "," + values2
assert node1.query(f"SELECT * FROM {TABLE_NAME} order by id FORMAT Values") == values1 + "," + values2
assert get_large_objects_count(blob_container_client) == 2

View File

@ -1053,3 +1053,22 @@ def table_table(clickhouse_node, mysql_node, service_name):
mysql_node.query("DROP DATABASE table_test")
clickhouse_node.query("DROP DATABASE table_test")
def table_overrides(clickhouse_node, mysql_node, service_name):
mysql_node.query("DROP DATABASE IF EXISTS table_overrides")
clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides")
mysql_node.query("CREATE DATABASE table_overrides")
mysql_node.query("CREATE TABLE table_overrides.t1 (sensor_id INT UNSIGNED, timestamp DATETIME, temperature FLOAT, PRIMARY KEY(timestamp, sensor_id))")
for id in range(10):
mysql_node.query("BEGIN")
for day in range(100):
mysql_node.query(f"INSERT INTO table_overrides.t1 VALUES({id}, TIMESTAMP('2021-01-01') + INTERVAL {day} DAY, (RAND()*20)+20)")
mysql_node.query("COMMIT")
clickhouse_node.query(f"""
CREATE DATABASE table_overrides ENGINE=MaterializeMySQL('{service_name}:3306', 'table_overrides', 'root', 'clickhouse')
TABLE OVERRIDE t1 (COLUMNS (sensor_id UInt64))
""")
check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1000\n")
check_query(clickhouse_node, "SELECT type FROM system.columns WHERE database = 'table_overrides' AND table = 't1' AND name = 'sensor_id'", "UInt64\n")
clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides")
mysql_node.query("DROP DATABASE IF EXISTS table_overrides")

View File

@ -249,3 +249,7 @@ def test_large_transaction(started_cluster, started_mysql_8_0, started_mysql_5_7
def test_table_table(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
materialize_with_ddl.table_table(clickhouse_node, started_mysql_8_0, "mysql80")
materialize_with_ddl.table_table(clickhouse_node, started_mysql_5_7, "mysql57")
def test_table_overrides(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
materialize_with_ddl.table_overrides(clickhouse_node, started_mysql_5_7, "mysql57")
materialize_with_ddl.table_overrides(clickhouse_node, started_mysql_8_0, "mysql80")

View File

@ -0,0 +1,5 @@
<clickhouse>
<background_processing_pool_thread_sleep_seconds>0.5</background_processing_pool_thread_sleep_seconds>
<background_processing_pool_task_sleep_seconds_when_no_work_min>0.5</background_processing_pool_task_sleep_seconds_when_no_work_min>
<background_processing_pool_task_sleep_seconds_when_no_work_max>0.5</background_processing_pool_task_sleep_seconds_when_no_work_max>
</clickhouse>

View File

@ -0,0 +1,33 @@
<clickhouse>
<storage_configuration>
<disks>
<blob_storage_disk>
<type>blob_storage</type>
<storage_account_url>http://azurite1:10000/devstoreaccount1</storage_account_url>
<container_name>cont</container_name>
<container_already_exists>false</container_already_exists>
<skip_access_check>false</skip_access_check>
<!-- default credentials for Azurite storage account -->
<account_name>devstoreaccount1</account_name>
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
<max_single_part_upload_size>33554432</max_single_part_upload_size>
</blob_storage_disk>
<hdd>
<type>local</type>
<path>/</path>
</hdd>
</disks>
<policies>
<blob_storage_policy>
<volumes>
<main>
<disk>blob_storage_disk</disk>
</main>
<external>
<disk>hdd</disk>
</external>
</volumes>
</blob_storage_policy>
</policies>
</storage_configuration>
</clickhouse>

View File

@ -0,0 +1,20 @@
<?xml version="1.0"?>
<clickhouse>
<tcp_port>9000</tcp_port>
<listen_host>127.0.0.1</listen_host>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
</clickhouse>

View File

@ -0,0 +1,344 @@
import logging
import time
import os
import pytest
from helpers.cluster import ClickHouseCluster, get_instances_dir
from helpers.utility import generate_values, replace_config, SafeThread
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_PATH = os.path.join(SCRIPT_DIR, './{}/node/configs/config.d/storage_conf.xml'.format(get_instances_dir()))
NODE_NAME = "node"
TABLE_NAME = "blob_storage_table"
BLOB_STORAGE_DISK = "blob_storage_disk"
LOCAL_DISK = "hdd"
CONTAINER_NAME = "cont"
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(NODE_NAME,
main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml"],
with_azurite=True)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def create_table(node, table_name, **additional_settings):
settings = {
"storage_policy": "blob_storage_policy",
"old_parts_lifetime": 1,
"index_granularity": 512
}
settings.update(additional_settings)
create_table_statement = f"""
CREATE TABLE {table_name} (
dt Date,
id Int64,
data String,
INDEX min_max (id) TYPE minmax GRANULARITY 3
) ENGINE=MergeTree()
PARTITION BY dt
ORDER BY (dt, id)
SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}"""
node.query(f"DROP TABLE IF EXISTS {table_name}")
node.query(create_table_statement)
assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)"
def test_create_table(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
def test_read_after_cache_is_wiped(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
values = "('2021-11-13',3,'hello'),('2021-11-14',4,'heyo')"
node.query(f"INSERT INTO {TABLE_NAME} VALUES {values}")
# Wipe cache
cluster.exec_in_container(cluster.get_container_id(NODE_NAME), ["rm", "-rf", "/var/lib/clickhouse/disks/blob_storage_disk/cache/"])
# After cache is populated again, only .bin files should be accessed from Blob Storage.
assert node.query(f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values
def test_simple_insert_select(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
values = "('2021-11-13',3,'hello')"
node.query(f"INSERT INTO {TABLE_NAME} VALUES {values}")
assert node.query(f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") == values
blob_container_client = cluster.blob_service_client.get_container_client(CONTAINER_NAME)
assert len(list(blob_container_client.list_blobs())) >= 12 # 1 format file + 2 skip index files + 9 regular MergeTree files + leftovers from other tests
def test_inserts_selects(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
values1 = generate_values('2020-01-03', 4096)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {values1}")
assert node.query(f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values1
values2 = generate_values('2020-01-04', 4096)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {values2}")
assert node.query(f"SELECT * FROM {TABLE_NAME} ORDER BY dt, id FORMAT Values") == values1 + "," + values2
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} where id = 1 FORMAT Values") == "(2)"
@pytest.mark.parametrize(
"merge_vertical", [
(True),
(False),
])
def test_insert_same_partition_and_merge(cluster, merge_vertical):
settings = {}
if merge_vertical:
settings['vertical_merge_algorithm_min_rows_to_activate'] = 0
settings['vertical_merge_algorithm_min_columns_to_activate'] = 0
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME, **settings)
node.query(f"SYSTEM STOP MERGES {TABLE_NAME}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024, -1)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048, -1)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"SYSTEM START MERGES {TABLE_NAME}")
# Wait for merges and old parts deletion
for attempt in range(0, 10):
parts_count = node.query(f"SELECT COUNT(*) FROM system.parts WHERE table = '{TABLE_NAME}' FORMAT Values")
if parts_count == "(1)":
break
if attempt == 9:
assert parts_count == "(1)"
time.sleep(1)
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
def test_alter_table_columns(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}")
node.query(f"ALTER TABLE {TABLE_NAME} ADD COLUMN col1 UInt64 DEFAULT 1")
# To ensure parts have been merged
node.query(f"OPTIMIZE TABLE {TABLE_NAME}")
assert node.query(f"SELECT sum(col1) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
assert node.query(f"SELECT sum(col1) FROM {TABLE_NAME} WHERE id > 0 FORMAT Values") == "(4096)"
node.query(f"ALTER TABLE {TABLE_NAME} MODIFY COLUMN col1 String", settings={"mutations_sync": 2})
assert node.query(f"SELECT distinct(col1) FROM {TABLE_NAME} FORMAT Values") == "('1')"
def test_attach_detach_partition(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"ALTER TABLE {TABLE_NAME} DETACH PARTITION '2020-01-03'")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)"
node.query(f"ALTER TABLE {TABLE_NAME} ATTACH PARTITION '2020-01-03'")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"ALTER TABLE {TABLE_NAME} DROP PARTITION '2020-01-03'")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)"
node.query(f"ALTER TABLE {TABLE_NAME} DETACH PARTITION '2020-01-04'")
node.query(f"ALTER TABLE {TABLE_NAME} DROP DETACHED PARTITION '2020-01-04'", settings={"allow_drop_detached": 1})
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)"
def test_move_partition_to_another_disk(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{LOCAL_DISK}'")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{BLOB_STORAGE_DISK}'")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
def test_table_manipulations(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
renamed_table = TABLE_NAME + "_renamed"
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}")
node.query(f"RENAME TABLE {TABLE_NAME} TO {renamed_table}")
assert node.query(f"SELECT count(*) FROM {renamed_table} FORMAT Values") == "(8192)"
node.query(f"RENAME TABLE {renamed_table} TO {TABLE_NAME}")
assert node.query(f"CHECK TABLE {TABLE_NAME} FORMAT Values") == "(1)"
node.query(f"DETACH TABLE {TABLE_NAME}")
node.query(f"ATTACH TABLE {TABLE_NAME}")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)"
node.query(f"TRUNCATE TABLE {TABLE_NAME}")
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)"
@pytest.mark.long_run
def test_move_replace_partition_to_another_table(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
table_clone_name = TABLE_NAME + "_clone"
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 256)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256, -1)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-06', 256, -1)}")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)"
create_table(node, table_clone_name)
node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-03' TO TABLE {table_clone_name}")
node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-05' TO TABLE {table_clone_name}")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(512)"
assert node.query(f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)"
# Add new partitions to source table, but with different values and replace them from copied table.
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256, -1)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256)}")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)"
node.query(f"ALTER TABLE {TABLE_NAME} REPLACE PARTITION '2020-01-03' FROM {table_clone_name}")
node.query(f"ALTER TABLE {TABLE_NAME} REPLACE PARTITION '2020-01-05' FROM {table_clone_name}")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)"
assert node.query(f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)"
node.query(f"DROP TABLE {table_clone_name} NO DELAY")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)"
node.query(f"ALTER TABLE {TABLE_NAME} FREEZE")
node.query(f"DROP TABLE {TABLE_NAME} NO DELAY")
def test_freeze_unfreeze(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
backup1 = 'backup1'
backup2 = 'backup2'
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
node.query(f"ALTER TABLE {TABLE_NAME} FREEZE WITH NAME '{backup1}'")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}")
node.query(f"ALTER TABLE {TABLE_NAME} FREEZE WITH NAME '{backup2}'")
node.query(f"TRUNCATE TABLE {TABLE_NAME}")
# Unfreeze single partition from backup1.
node.query(f"ALTER TABLE {TABLE_NAME} UNFREEZE PARTITION '2020-01-03' WITH NAME '{backup1}'")
# Unfreeze all partitions from backup2.
node.query(f"ALTER TABLE {TABLE_NAME} UNFREEZE WITH NAME '{backup2}'")
def test_apply_new_settings(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}")
# Force multi-part upload mode.
replace_config(
CONFIG_PATH,
"<max_single_part_upload_size>33554432</max_single_part_upload_size>",
"<max_single_part_upload_size>4096</max_single_part_upload_size>")
node.query("SYSTEM RELOAD CONFIG")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096, -1)}")
# NOTE: this test takes a couple of minutes when run together with other tests
@pytest.mark.long_run
def test_restart_during_load(cluster):
node = cluster.instances[NODE_NAME]
create_table(node, TABLE_NAME)
# Force multi-part upload mode.
replace_config(CONFIG_PATH, "<container_already_exists>false</container_already_exists>", "")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}")
node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 4096, -1)}")
def read():
for ii in range(0, 5):
logging.info(f"Executing {ii} query")
assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)"
logging.info(f"Query {ii} executed")
time.sleep(0.2)
def restart_disk():
for iii in range(0, 2):
logging.info(f"Restarting disk, attempt {iii}")
node.query(f"SYSTEM RESTART DISK {BLOB_STORAGE_DISK}")
logging.info(f"Disk restarted, attempt {iii}")
time.sleep(0.5)
threads = []
for _ in range(0, 4):
threads.append(SafeThread(target=read))
threads.append(SafeThread(target=restart_disk))
for thread in threads:
thread.start()
for thread in threads:
thread.join()

View File

@ -1,12 +1,10 @@
import logging
import random
import string
import time
import threading
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.utility import generate_values
from pyhdfs import HdfsClient
@ -43,17 +41,6 @@ FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
def random_string(length):
letters = string.ascii_letters
return ''.join(random.choice(letters) for i in range(length))
def generate_values(date_str, count, sign=1):
data = [[date_str, sign * (i + 1), random_string(10)] for i in range(count)]
data.sort(key=lambda tup: tup[1])
return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data])
@pytest.fixture(scope="module")
def cluster():
try:

View File

@ -1,47 +1,16 @@
import logging
import random
import string
import time
import threading
import os
import pytest
from helpers.cluster import ClickHouseCluster, get_instances_dir
from helpers.utility import generate_values, replace_config, SafeThread
# By default the exceptions that was throwed in threads will be ignored
# (they will not mark the test as failed, only printed to stderr).
#
# Wrap thrading.Thread and re-throw exception on join()
class SafeThread(threading.Thread):
def __init__(self, target):
super().__init__()
self.target = target
self.exception = None
def run(self):
try:
self.target()
except Exception as e: # pylint: disable=broad-except
self.exception = e
def join(self, timeout=None):
super().join(timeout)
if self.exception:
raise self.exception
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_PATH = os.path.join(SCRIPT_DIR, './{}/node/configs/config.d/storage_conf.xml'.format(get_instances_dir()))
def replace_config(old, new):
config = open(CONFIG_PATH, 'r')
config_lines = config.readlines()
config.close()
config_lines = [line.replace(old, new) for line in config_lines]
config = open(CONFIG_PATH, 'w')
config.writelines(config_lines)
config.close()
@pytest.fixture(scope="module")
def cluster():
try:
@ -66,17 +35,6 @@ FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
def random_string(length):
letters = string.ascii_letters
return ''.join(random.choice(letters) for i in range(length))
def generate_values(date_str, count, sign=1):
data = [[date_str, sign * (i + 1), random_string(10)] for i in range(count)]
data.sort(key=lambda tup: tup[1])
return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data])
def create_table(node, table_name, **additional_settings):
settings = {
"storage_policy": "s3",
@ -442,8 +400,9 @@ def test_s3_disk_apply_new_settings(cluster, node_name):
s3_requests_to_write_partition = get_s3_requests() - s3_requests_before
# Force multi-part upload mode.
replace_config("<s3_max_single_part_upload_size>33554432</s3_max_single_part_upload_size>",
"<s3_max_single_part_upload_size>0</s3_max_single_part_upload_size>")
replace_config(CONFIG_PATH,
"<s3_max_single_part_upload_size>33554432</s3_max_single_part_upload_size>",
"<s3_max_single_part_upload_size>0</s3_max_single_part_upload_size>")
node.query("SYSTEM RELOAD CONFIG")

View File

@ -46,6 +46,20 @@ def wait_for_large_objects_count(cluster, expected, size=100, timeout=30):
assert get_large_objects_count(cluster, size=size) == expected
def wait_for_active_parts(node, num_expected_parts, table_name, timeout=30):
deadline = time.monotonic() + timeout
num_parts = 0
while time.monotonic() < deadline:
num_parts_str = node.query("select count() from system.parts where table = '{}' and active".format(table_name))
num_parts = int(num_parts_str.strip())
if num_parts == num_expected_parts:
return
time.sleep(0.2)
assert num_parts == num_expected_parts
@pytest.mark.parametrize(
"policy", ["s3"]
)
@ -248,3 +262,50 @@ def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations):
node1.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
def test_s3_zero_copy_concurrent_merge(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
node1.query("DROP TABLE IF EXISTS concurrent_merge NO DELAY")
node2.query("DROP TABLE IF EXISTS concurrent_merge NO DELAY")
for node in (node1, node2):
node.query(
"""
CREATE TABLE concurrent_merge (id UInt64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/concurrent_merge', '{replica}')
ORDER BY id
SETTINGS index_granularity=2, storage_policy='s3', remote_fs_execute_merges_on_single_replica_time_threshold=1
"""
)
node1.query("system stop merges")
node2.query("system stop merges")
# This will generate two parts with 20 granules each
node1.query("insert into concurrent_merge select number from numbers(40)")
node1.query("insert into concurrent_merge select number + 1 from numbers(40)")
wait_for_active_parts(node2, 2, 'concurrent_merge')
# Merge will materialize default column, it should sleep every granule and take 20 * 2 * 0.1 = 4 sec.
node1.query("alter table concurrent_merge add column x UInt32 default sleep(0.1)")
node1.query("system start merges")
node2.query("system start merges")
# Now, the merge should start.
# Because of remote_fs_execute_merges_on_single_replica_time_threshold=1,
# only one replica will start merge instantly.
# The other replica should wait for 1 sec and also start it.
# That should probably cause a data race at s3 storage.
# For now, it does not happen (every blob has a random name, and we just have a duplicating data)
node1.query("optimize table concurrent_merge final")
wait_for_active_parts(node1, 1, 'concurrent_merge')
wait_for_active_parts(node2, 1, 'concurrent_merge')
for node in (node1, node2):
assert node.query('select sum(id) from concurrent_merge').strip() == '1600'

View File

@ -159,3 +159,27 @@ def test_no_credentials(started_cluster):
node.query("create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')")
assert node.query("SELECT count() FROM simple_mongo_table_2") == '100\n'
simple_mongo_table.drop()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_auth_source(started_cluster):
mongo_connection = get_mongo_connection(started_cluster, with_credentials=False)
admin_db = mongo_connection['admin']
admin_db.add_user('root', 'clickhouse', roles=[{ 'role': "userAdminAnyDatabase", 'db': "admin" }, "readWriteAnyDatabase"])
simple_mongo_table = admin_db['simple_table']
data = []
for i in range(0, 50):
data.append({'key': i, 'data': hex(i * i)})
simple_mongo_table.insert_many(data)
db = mongo_connection['test']
simple_mongo_table = db['simple_table']
data = []
for i in range(0, 100):
data.append({'key': i, 'data': hex(i * i)})
simple_mongo_table.insert_many(data)
node = started_cluster.instances['node']
node.query("create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse')")
node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail")
node.query("create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')")
assert node.query("SELECT count() FROM simple_mongo_table_ok") == '100\n'
simple_mongo_table.drop()

View File

@ -69,6 +69,17 @@ DateTime64(3, \'Europe/Moscow\') DateTime64(6, \'Europe/Moscow\') DateTime64(6,
1989-12-13 02:01:16.532 1992-10-05 07:07:57.973222 2037-10-24 18:53:50.985504
1992-12-28 12:26:04.030 1971-07-29 09:20:38.230976 1980-03-26 18:49:55.428516
2051-12-11 10:09:13.162 1982-01-12 03:25:45.754492 2010-05-17 11:01:28.452864
Date32
1934-01-06
2039-08-16
2103-11-03
2064-08-14
2187-08-21
2099-04-08
1947-06-22
2012-01-19
2170-07-09
2263-01-17
-
Float32 Float64
-1.3551149e32 1.2262973812461839e235

View File

@ -57,6 +57,14 @@ SELECT
dt64, dts64, dtms64
FROM generateRandom('dt64 DateTime64(3, \'Europe/Moscow\'), dts64 DateTime64(6, \'Europe/Moscow\'), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 1, 10, 10)
LIMIT 10;
SELECT
toTypeName(d32)
FROM generateRandom('d32 Date32')
LIMIT 1;
SELECT
d32
FROM generateRandom('d32 Date32', 1, 10, 10)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(f32), toTypeName(f64)

View File

@ -1,9 +1,9 @@
===http===
{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"SELECT 1 FROM system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"SELECT 1 FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"SELECT 1 FROM system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"SELECT 1 FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"}
{"initial query spans with proper parent":"1"}

View File

@ -8,11 +8,11 @@ ccccccccc aaaaaaaaa bbbbbbbbb
aaaaaaaaa
bbbbbbbbb
ccccccccc
:233
:79
:107
:74
:35
:35
:35
:233
695071
aaaaaaaaa bbbbbbbbb
ccccccccc aaaaaaaaa bbbbbbbbb
@ -21,5 +21,9 @@ ccccccccc aaaaaaaaa bbbbbbbbb
ccccccccc aaaaaaaaa bbbbbbbbb
695071 0
:0
:233
:79
:107
:74
Hello 2
World 1
Hello 2
World 1

View File

@ -8,16 +8,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
# Data preparation.
# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as:
# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')"
user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
mkdir -p ${user_files_path}/
echo -n aaaaaaaaa > ${user_files_path}/a.txt
echo -n bbbbbbbbb > ${user_files_path}/b.txt
echo -n ccccccccc > ${user_files_path}/c.txt
mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/
echo -n aaaaaaaaa > ${CLICKHOUSE_USER_FILES_PATH}/a.txt
echo -n bbbbbbbbb > ${CLICKHOUSE_USER_FILES_PATH}/b.txt
echo -n ccccccccc > ${CLICKHOUSE_USER_FILES_PATH}/c.txt
echo -n ccccccccc > /tmp/c.txt
mkdir -p ${user_files_path}/dir
mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/dir
### 1st TEST in CLIENT mode.
@ -26,28 +27,28 @@ ${CLICKHOUSE_CLIENT} --query "create table data (A String, B String) engine=Merg
# Valid cases:
${CLICKHOUSE_CLIENT} --query "select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "insert into data select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "insert into data select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "select file('${user_files_path}/c.txt'), * from data";echo ":"$?
${CLICKHOUSE_CLIENT} --query "select file('a.txt'), file('b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$?
${CLICKHOUSE_CLIENT} --query "select file('c.txt'), * from data";echo ":"$?
${CLICKHOUSE_CLIENT} --multiquery --query "
create table filenames(name String) engine=MergeTree() order by tuple();
insert into filenames values ('a.txt'), ('b.txt'), ('c.txt');
select file(name) from filenames format TSV;
drop table if exists filenames;
create table filenames(name String) engine=MergeTree() order by tuple();
insert into filenames values ('a.txt'), ('b.txt'), ('c.txt');
select file(name) from filenames format TSV;
drop table if exists filenames;
"
# Invalid cases: (Here using sub-shell to catch exception avoiding the test quit)
# Test non-exists file
echo "clickhouse-client --query "'"select file('"'nonexist.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
# Test isDir
echo "clickhouse-client --query "'"select file('"'${user_files_path}/dir'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
# Test path out of the user_files directory. It's not allowed in client mode
echo "clickhouse-client --query "'"select file('"'/tmp/c.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'/tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
# Test relative path consists of ".." whose absolute path is out of the user_files directory.
echo "clickhouse-client --query "'"select file('"'${user_files_path}/../../../../../../../../../../../../../../../../../../../tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
echo "clickhouse-client --query "'"select file('"'../../../../a.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'../../../../../../../../../../../../../../../../../../../tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'../../../../a.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
### 2nd TEST in LOCAL mode.
@ -63,29 +64,36 @@ echo $c_count
# Valid cases:
# The default dir is the CWD path in LOCAL mode
${CLICKHOUSE_LOCAL} --query "
drop table if exists data;
create table data (A String, B String) engine=MergeTree() order by A;
select file('a.txt'), file('b.txt');
insert into data select file('a.txt'), file('b.txt');
insert into data select file('a.txt'), file('b.txt');
select file('c.txt'), * from data;
select file('/tmp/c.txt'), * from data;
select $c_count, $c_count -length(file('${CURDIR}/01518_nullable_aggregate_states2.reference'))
drop table if exists data;
create table data (A String, B String) engine=MergeTree() order by A;
select file('a.txt'), file('b.txt');
insert into data select file('a.txt'), file('b.txt');
insert into data select file('a.txt'), file('b.txt');
select file('c.txt'), * from data;
select file('/tmp/c.txt'), * from data;
select $c_count, $c_count -length(file('${CURDIR}/01518_nullable_aggregate_states2.reference'))
"
echo ":"$?
# Invalid cases: (Here using sub-shell to catch exception avoiding the test quit)
# Test non-exists file
echo "clickhouse-local --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_LOCAL} --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
# Test isDir
echo "clickhouse-local --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
echo "${CLICKHOUSE_LOCAL} --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
# Test that the function is not injective
echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/a
echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/b
echo -n World > ${CLICKHOUSE_USER_FILES_PATH}/c
${CLICKHOUSE_CLIENT} --query "SELECT file(arrayJoin(['a', 'b', 'c'])) AS s, count() GROUP BY s ORDER BY s"
${CLICKHOUSE_CLIENT} --query "SELECT s, count() FROM file('?', TSV, 's String') GROUP BY s ORDER BY s"
# Restore
rm -rf a.txt b.txt c.txt dir
rm -rf ${user_files_path}/a.txt
rm -rf ${user_files_path}/b.txt
rm -rf ${user_files_path}/c.txt
rm -rf /tmp/c.txt
rm -rf ${user_files_path}/dir
rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c}.txt
rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c}
rm /tmp/c.txt
rm -rf ${CLICKHOUSE_USER_FILES_PATH}/dir

View File

@ -1,17 +1,17 @@
(0, 2)
0 0
0 0
WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2)
WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_no` SELECT `one`.`dummy`, ignore(`id_no`) FROM `system`.`one` WHERE `dummy` IN (0, 2)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_no` SELECT `one`.`dummy`, ignore(`id_no`) FROM `system`.`one` WHERE `dummy` IN (0, 2)
optimize_skip_unused_shards_rewrite_in(0, 2)
0 0
WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(0)
WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(2)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_02` SELECT `one`.`dummy`, ignore(`id_02`) FROM `system`.`one` WHERE `dummy` IN tuple(0)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_02` SELECT `one`.`dummy`, ignore(`id_02`) FROM `system`.`one` WHERE `dummy` IN tuple(2)
optimize_skip_unused_shards_rewrite_in(2,)
WITH _CAST(\'default\', \'Nullable(String)\') AS id_2 SELECT one.dummy, ignore(id_2) FROM system.one WHERE dummy IN tuple(2)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_2` SELECT `one`.`dummy`, ignore(`id_2`) FROM `system`.`one` WHERE `dummy` IN tuple(2)
optimize_skip_unused_shards_rewrite_in(0,)
0 0
WITH _CAST(\'default\', \'Nullable(String)\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0)
WITH _CAST(\'default\', \'Nullable(String)\') AS `id_0` SELECT `one`.`dummy`, ignore(`id_0`) FROM `system`.`one` WHERE `dummy` IN tuple(0)
0
0
errors

Some files were not shown because too many files have changed in this diff Show More