mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into merging_template_format
This commit is contained in:
commit
1572d0c5b6
@ -264,11 +264,10 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||
endif ()
|
||||
|
||||
# TODO: only make this extra-checks in CI builds, since a lot of contrib libs won't link -
|
||||
# CI works around this problem by explicitly adding GLIBC_COMPATIBILITY flag.
|
||||
if (NOT SANITIZE AND YANDEX_OFFICIAL_BUILD)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
# Make this extra-checks for correct library dependencies.
|
||||
if (NOT SANITIZE)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
endif ()
|
||||
|
||||
include (cmake/find_unwind.cmake)
|
||||
|
@ -219,7 +219,7 @@ endif()
|
||||
add_library(${ARROW_LIBRARY} ${ARROW_SRCS})
|
||||
add_dependencies(${ARROW_LIBRARY} protoc)
|
||||
target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS})
|
||||
target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} Threads::Threads)
|
||||
target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY} Threads::Threads)
|
||||
if (ARROW_WITH_LZ4)
|
||||
target_link_libraries(${ARROW_LIBRARY} PRIVATE ${LZ4_LIBRARY})
|
||||
endif()
|
||||
|
@ -114,6 +114,7 @@ add_headers_and_sources(dbms src/Columns)
|
||||
add_headers_and_sources(dbms src/Storages)
|
||||
add_headers_and_sources(dbms src/Storages/Distributed)
|
||||
add_headers_and_sources(dbms src/Storages/MergeTree)
|
||||
add_headers_and_sources(dbms src/Storages/LiveView)
|
||||
add_headers_and_sources(dbms src/Client)
|
||||
add_headers_and_sources(dbms src/Formats)
|
||||
add_headers_and_sources(dbms src/Processors)
|
||||
|
@ -217,6 +217,7 @@
|
||||
|
||||
See https://clickhouse.yandex/docs/en/table_engines/replication/
|
||||
-->
|
||||
|
||||
<zookeeper incl="zookeeper-servers" optional="true" />
|
||||
|
||||
<!-- Substitutions for parameters of replicated tables.
|
||||
|
@ -61,43 +61,18 @@ std::vector<size_t> countColumnsSizeInSelector(IColumn::ColumnIndex num_columns,
|
||||
return counts;
|
||||
}
|
||||
|
||||
/** clang 4 generates better code than gcc 6.
|
||||
* And both gcc and clang could not vectorize trivial loop by bytes automatically.
|
||||
*/
|
||||
bool memoryIsZero(const void * data, size_t size)
|
||||
bool memoryIsByte(const void * data, size_t size, uint8_t byte)
|
||||
{
|
||||
const Int8 * pos = reinterpret_cast<const Int8 *>(data);
|
||||
const Int8 * end = pos + size;
|
||||
|
||||
#ifdef __SSE2__
|
||||
const __m128 zero16 = _mm_setzero_ps();
|
||||
const Int8 * end64 = pos + size / 64 * 64;
|
||||
|
||||
for (; pos < end64; pos += 64)
|
||||
if (_mm_movemask_ps(_mm_cmpneq_ps(
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(pos)),
|
||||
zero16))
|
||||
| _mm_movemask_ps(_mm_cmpneq_ps(
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(pos + 16)),
|
||||
zero16))
|
||||
| _mm_movemask_ps(_mm_cmpneq_ps(
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(pos + 32)),
|
||||
zero16))
|
||||
| _mm_movemask_ps(_mm_cmpneq_ps(
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(pos + 48)),
|
||||
zero16)))
|
||||
return false;
|
||||
|
||||
/// TODO Add duff device for tail?
|
||||
#endif
|
||||
|
||||
for (; pos < end; ++pos)
|
||||
if (*pos)
|
||||
return false;
|
||||
|
||||
return true;
|
||||
if (size == 0)
|
||||
return true;
|
||||
auto ptr = reinterpret_cast<const uint8_t *>(data);
|
||||
return *ptr == byte && memcmp(ptr, ptr + 1, size - 1) == 0;
|
||||
}
|
||||
|
||||
bool memoryIsZero(const void * data, size_t size)
|
||||
{
|
||||
return memoryIsByte(data, size, 0x0);
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ std::vector<size_t> countColumnsSizeInSelector(IColumn::ColumnIndex num_columns,
|
||||
|
||||
/// Returns true, if the memory contains only zeros.
|
||||
bool memoryIsZero(const void * data, size_t size);
|
||||
|
||||
bool memoryIsByte(const void * data, size_t size, uint8_t byte);
|
||||
|
||||
/// The general implementation of `filter` function for ColumnArray and ColumnString.
|
||||
template <typename T>
|
||||
|
@ -443,8 +443,9 @@ namespace ErrorCodes
|
||||
extern const int INSECURE_PATH = 466;
|
||||
extern const int CANNOT_PARSE_BOOL = 467;
|
||||
extern const int CANNOT_PTHREAD_ATTR = 468;
|
||||
extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 469;
|
||||
extern const int INVALID_TEMPLATE_FORMAT = 470;
|
||||
extern const int VIOLATED_CONSTRAINT = 469;
|
||||
extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470;
|
||||
extern const int INVALID_TEMPLATE_FORMAT = 471;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -33,8 +33,6 @@
|
||||
#define DEFAULT_MERGE_BLOCK_SIZE 8192
|
||||
|
||||
#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5
|
||||
#define DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC 15
|
||||
#define DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS 10000
|
||||
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
|
||||
#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15
|
||||
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
|
||||
|
@ -335,7 +335,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \
|
||||
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \
|
||||
\
|
||||
M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \
|
||||
M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \
|
||||
\
|
||||
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
|
||||
M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \
|
||||
@ -343,16 +343,16 @@ struct Settings : public SettingsCollection<Settings>
|
||||
\
|
||||
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \
|
||||
M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \
|
||||
M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries") \
|
||||
\
|
||||
M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.") \
|
||||
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \
|
||||
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \
|
||||
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \
|
||||
\
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
\
|
||||
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \
|
||||
\
|
||||
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \
|
||||
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \
|
||||
M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \
|
||||
M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") \
|
||||
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \
|
||||
|
||||
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)
|
||||
|
||||
|
79
dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp
Normal file
79
dbms/src/DataStreams/CheckConstraintsBlockOutputStream.cpp
Normal file
@ -0,0 +1,79 @@
|
||||
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void CheckConstraintsBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
for (size_t i = 0; i < expressions.size(); ++i)
|
||||
{
|
||||
Block res = block;
|
||||
auto constraint_expr = expressions[i];
|
||||
auto res_column_uint8 = executeOnBlock(res, constraint_expr);
|
||||
if (!memoryIsByte(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize(), 0x1))
|
||||
{
|
||||
auto indices_wrong = findAllWrong(res_column_uint8->getRawDataBegin<1>(), res_column_uint8->byteSize());
|
||||
std::string indices_str = "{";
|
||||
for (size_t j = 0; j < indices_wrong.size(); ++j)
|
||||
{
|
||||
indices_str += std::to_string(indices_wrong[j]);
|
||||
indices_str += (j != indices_wrong.size() - 1) ? ", " : "}";
|
||||
}
|
||||
|
||||
throw Exception{"Violated constraint " + constraints.constraints[i]->name +
|
||||
" in table " + table + " at indices " + indices_str + ", constraint expression: " +
|
||||
serializeAST(*(constraints.constraints[i]->expr), true), ErrorCodes::VIOLATED_CONSTRAINT};
|
||||
}
|
||||
}
|
||||
output->write(block);
|
||||
rows_written += block.rows();
|
||||
}
|
||||
|
||||
void CheckConstraintsBlockOutputStream::flush()
|
||||
{
|
||||
output->flush();
|
||||
}
|
||||
|
||||
void CheckConstraintsBlockOutputStream::writePrefix()
|
||||
{
|
||||
output->writePrefix();
|
||||
}
|
||||
|
||||
void CheckConstraintsBlockOutputStream::writeSuffix()
|
||||
{
|
||||
output->writeSuffix();
|
||||
}
|
||||
|
||||
const ColumnUInt8 *CheckConstraintsBlockOutputStream::executeOnBlock(
|
||||
Block & block,
|
||||
const ExpressionActionsPtr & constraint)
|
||||
{
|
||||
constraint->execute(block);
|
||||
ColumnWithTypeAndName res_column = block.safeGetByPosition(block.columns() - 1);
|
||||
return checkAndGetColumn<ColumnUInt8>(res_column.column.get());
|
||||
}
|
||||
|
||||
std::vector<size_t> CheckConstraintsBlockOutputStream::findAllWrong(const void *data, size_t size)
|
||||
{
|
||||
std::vector<size_t> res;
|
||||
|
||||
if (size == 0)
|
||||
return res;
|
||||
|
||||
auto ptr = reinterpret_cast<const uint8_t *>(data);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (*(ptr + i) == 0x0)
|
||||
{
|
||||
res.push_back(i);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
}
|
52
dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h
Normal file
52
dbms/src/DataStreams/CheckConstraintsBlockOutputStream.h
Normal file
@ -0,0 +1,52 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int VIOLATED_CONSTRAINT;
|
||||
}
|
||||
|
||||
class CheckConstraintsBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
CheckConstraintsBlockOutputStream(
|
||||
const String & table_,
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_)
|
||||
: table(table_),
|
||||
output(output_),
|
||||
header(header_),
|
||||
constraints(constraints_),
|
||||
expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())),
|
||||
rows_written(0)
|
||||
{ }
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
void write(const Block & block) override;
|
||||
|
||||
void flush() override;
|
||||
|
||||
void writePrefix() override;
|
||||
void writeSuffix() override;
|
||||
|
||||
private:
|
||||
const ColumnUInt8* executeOnBlock(Block & block, const ExpressionActionsPtr & constraint);
|
||||
std::vector<size_t> findAllWrong(const void *data, size_t size);
|
||||
|
||||
String table;
|
||||
BlockOutputStreamPtr output;
|
||||
Block header;
|
||||
const ConstraintsDescription constraints;
|
||||
const ConstraintsExpressions expressions;
|
||||
size_t rows_written;
|
||||
};
|
||||
}
|
@ -12,7 +12,7 @@
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -106,8 +106,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
|
||||
if (auto * live_view = dynamic_cast<StorageLiveView *>(storage.get()))
|
||||
{
|
||||
BlockOutputStreamPtr output_ = std::make_shared<LiveViewBlockOutputStream>(*live_view);
|
||||
StorageLiveView::writeIntoLiveView(*live_view, block, context, output_);
|
||||
StorageLiveView::writeIntoLiveView(*live_view, block, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -129,6 +129,7 @@ void DatabaseDictionary::alterTable(
|
||||
const String &,
|
||||
const ColumnsDescription &,
|
||||
const IndicesDescription &,
|
||||
const ConstraintsDescription &,
|
||||
const ASTModifier &)
|
||||
{
|
||||
throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
@ -71,6 +71,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -53,6 +53,7 @@ void DatabaseMemory::alterTable(
|
||||
const String &,
|
||||
const ColumnsDescription &,
|
||||
const IndicesDescription &,
|
||||
const ConstraintsDescription &,
|
||||
const ASTModifier &)
|
||||
{
|
||||
throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
@ -48,6 +48,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -71,7 +71,7 @@ public:
|
||||
throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ASTModifier &) override
|
||||
void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ConstraintsDescription &, const ASTModifier &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support alter table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -487,6 +487,7 @@ void DatabaseOrdinary::alterTable(
|
||||
const String & table_name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & storage_modifier)
|
||||
{
|
||||
/// Read the definition of the table and replace the necessary parts with new ones.
|
||||
@ -509,6 +510,7 @@ void DatabaseOrdinary::alterTable(
|
||||
|
||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(indices);
|
||||
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(constraints);
|
||||
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
|
||||
|
||||
@ -517,6 +519,11 @@ void DatabaseOrdinary::alterTable(
|
||||
else
|
||||
ast_create_query.columns_list->set(ast_create_query.columns_list->indices, new_indices);
|
||||
|
||||
if (ast_create_query.columns_list->constraints)
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->constraints, new_constraints);
|
||||
else
|
||||
ast_create_query.columns_list->set(ast_create_query.columns_list->constraints, new_constraints);
|
||||
|
||||
if (storage_modifier)
|
||||
storage_modifier(*ast_create_query.storage);
|
||||
|
||||
|
@ -42,6 +42,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
@ -115,6 +116,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) = 0;
|
||||
|
||||
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
|
||||
|
194
dbms/src/Functions/neighbor.cpp
Normal file
194
dbms/src/Functions/neighbor.cpp
Normal file
@ -0,0 +1,194 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
// Implements function, giving value for column within range of given
|
||||
// Example:
|
||||
// | c1 |
|
||||
// | 10 |
|
||||
// | 20 |
|
||||
// SELECT c1, neighbor(c1, 1) as c2:
|
||||
// | c1 | c2 |
|
||||
// | 10 | 20 |
|
||||
// | 20 | 0 |
|
||||
class FunctionNeighbor : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "neighbor";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionNeighbor>(context); }
|
||||
|
||||
FunctionNeighbor(const Context & context_) : context(context_) {}
|
||||
|
||||
/// Get the name of the function.
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
size_t number_of_arguments = arguments.size();
|
||||
|
||||
if (number_of_arguments < 2 || number_of_arguments > 3)
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments)
|
||||
+ ", should be from 2 to 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
// second argument must be an integer
|
||||
if (!isInteger(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - should be an integer",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else if (arguments[1]->isNullable())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - can not be Nullable",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
// check that default value column has supertype with first argument
|
||||
if (number_of_arguments == 3)
|
||||
return getLeastSupertype({arguments[0], arguments[2]});
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const DataTypePtr & result_type = block.getByPosition(result).type;
|
||||
|
||||
const ColumnWithTypeAndName & source_elem = block.getByPosition(arguments[0]);
|
||||
const ColumnWithTypeAndName & offset_elem = block.getByPosition(arguments[1]);
|
||||
bool has_defaults = arguments.size() == 3;
|
||||
|
||||
ColumnPtr source_column_casted = castColumn(source_elem, result_type, context);
|
||||
ColumnPtr offset_column = offset_elem.column;
|
||||
|
||||
ColumnPtr default_column_casted;
|
||||
if (has_defaults)
|
||||
{
|
||||
const ColumnWithTypeAndName & default_elem = block.getByPosition(arguments[2]);
|
||||
default_column_casted = castColumn(default_elem, result_type, context);
|
||||
}
|
||||
|
||||
bool source_is_constant = isColumnConst(*source_column_casted);
|
||||
bool offset_is_constant = isColumnConst(*offset_column);
|
||||
|
||||
bool default_is_constant = false;
|
||||
if (has_defaults)
|
||||
default_is_constant = isColumnConst(*default_column_casted);
|
||||
|
||||
if (source_is_constant)
|
||||
source_column_casted = assert_cast<const ColumnConst &>(*source_column_casted).getDataColumnPtr();
|
||||
if (offset_is_constant)
|
||||
offset_column = assert_cast<const ColumnConst &>(*offset_column).getDataColumnPtr();
|
||||
if (default_is_constant)
|
||||
default_column_casted = assert_cast<const ColumnConst &>(*default_column_casted).getDataColumnPtr();
|
||||
|
||||
if (offset_is_constant)
|
||||
{
|
||||
/// Optimization for the case when we can copy many values at once.
|
||||
|
||||
Int64 offset = offset_column->getInt(0);
|
||||
|
||||
auto result_column = result_type->createColumn();
|
||||
|
||||
auto insert_range_from = [&](bool is_const, const ColumnPtr & src, Int64 begin, Int64 size)
|
||||
{
|
||||
/// Saturation of bounds.
|
||||
if (begin < 0)
|
||||
{
|
||||
size += begin;
|
||||
begin = 0;
|
||||
}
|
||||
if (size <= 0)
|
||||
return;
|
||||
if (size > Int64(input_rows_count))
|
||||
size = input_rows_count;
|
||||
|
||||
if (!src)
|
||||
{
|
||||
for (Int64 i = 0; i < size; ++i)
|
||||
result_column->insertDefault();
|
||||
}
|
||||
else if (is_const)
|
||||
{
|
||||
for (Int64 i = 0; i < size; ++i)
|
||||
result_column->insertFrom(*src, 0);
|
||||
}
|
||||
else
|
||||
{
|
||||
result_column->insertRangeFrom(*src, begin, size);
|
||||
}
|
||||
};
|
||||
|
||||
if (offset == 0)
|
||||
{
|
||||
/// Degenerate case, just copy source column as is.
|
||||
block.getByPosition(result).column = source_is_constant ? ColumnConst::create(source_column_casted, input_rows_count) : source_column_casted;
|
||||
}
|
||||
else if (offset > 0)
|
||||
{
|
||||
insert_range_from(source_is_constant, source_column_casted, offset, Int64(input_rows_count) - offset);
|
||||
insert_range_from(default_is_constant, default_column_casted, Int64(input_rows_count) - offset, offset);
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
else
|
||||
{
|
||||
insert_range_from(default_is_constant, default_column_casted, 0, -offset);
|
||||
insert_range_from(source_is_constant, source_column_casted, 0, Int64(input_rows_count) + offset);
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto result_column = result_type->createColumn();
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
Int64 src_idx = row + offset_column->getInt(offset_is_constant ? 0 : row);
|
||||
|
||||
if (src_idx >= 0 && src_idx < Int64(input_rows_count))
|
||||
result_column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx);
|
||||
else if (has_defaults)
|
||||
result_column->insertFrom(*default_column_casted, default_is_constant ? 0 : row);
|
||||
else
|
||||
result_column->insertDefault();
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
void registerFunctionNeighbor(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionNeighbor>();
|
||||
}
|
||||
|
||||
}
|
@ -18,6 +18,7 @@ void registerFunctionBlockSize(FunctionFactory &);
|
||||
void registerFunctionBlockNumber(FunctionFactory &);
|
||||
void registerFunctionRowNumberInBlock(FunctionFactory &);
|
||||
void registerFunctionRowNumberInAllBlocks(FunctionFactory &);
|
||||
void registerFunctionNeighbor(FunctionFactory &);
|
||||
void registerFunctionSleep(FunctionFactory &);
|
||||
void registerFunctionSleepEachRow(FunctionFactory &);
|
||||
void registerFunctionMaterialize(FunctionFactory &);
|
||||
@ -69,6 +70,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionBlockNumber(factory);
|
||||
registerFunctionRowNumberInBlock(factory);
|
||||
registerFunctionRowNumberInAllBlocks(factory);
|
||||
registerFunctionNeighbor(factory);
|
||||
registerFunctionSleep(factory);
|
||||
registerFunctionSleepEachRow(factory);
|
||||
registerFunctionMaterialize(factory);
|
||||
|
@ -54,6 +54,7 @@
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
||||
#include <Interpreters/ExternalTablesVisitor.h>
|
||||
|
@ -8,9 +8,9 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <Storages/LiveViewCommands.h>
|
||||
#include <Storages/LiveView/LiveViewCommands.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
|
||||
#include <algorithm>
|
||||
|
||||
@ -22,6 +22,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
|
||||
@ -56,7 +57,13 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
if (auto alter_command = AlterCommand::parse(command_ast))
|
||||
alter_commands.emplace_back(std::move(*alter_command));
|
||||
else if (auto partition_command = PartitionCommand::parse(command_ast))
|
||||
{
|
||||
if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION
|
||||
&& !context.getSettingsRef().allow_drop_detached)
|
||||
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
|
||||
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
partition_commands.emplace_back(std::move(*partition_command));
|
||||
}
|
||||
else if (auto mut_command = MutationCommand::parse(command_ast))
|
||||
mutation_commands.emplace_back(std::move(*mut_command));
|
||||
else if (auto live_view_command = LiveViewCommand::parse(command_ast))
|
||||
|
@ -252,6 +252,16 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices)
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints)
|
||||
{
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & constraint : constraints.constraints)
|
||||
res->children.push_back(constraint->clone());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns_ast, const Context & context)
|
||||
{
|
||||
/// First, deduce implicit types.
|
||||
@ -376,6 +386,8 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
|
||||
{
|
||||
ColumnsDescription columns;
|
||||
IndicesDescription indices;
|
||||
ConstraintsDescription constraints;
|
||||
|
||||
|
||||
if (create.columns_list)
|
||||
{
|
||||
@ -385,11 +397,16 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
|
||||
for (const auto & index : create.columns_list->indices->children)
|
||||
indices.indices.push_back(
|
||||
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
|
||||
if (create.columns_list->constraints)
|
||||
for (const auto & constraint : create.columns_list->constraints->children)
|
||||
constraints.constraints.push_back(
|
||||
std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint->clone()));
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
{
|
||||
columns = as_storage->getColumns();
|
||||
indices = as_storage->getIndices();
|
||||
constraints = as_storage->getConstraints();
|
||||
}
|
||||
else if (create.select)
|
||||
{
|
||||
@ -401,6 +418,7 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
|
||||
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
|
||||
ASTPtr new_columns = formatColumns(columns);
|
||||
ASTPtr new_indices = formatIndices(indices);
|
||||
ASTPtr new_constraints = formatConstraints(constraints);
|
||||
|
||||
if (!create.columns_list)
|
||||
{
|
||||
@ -418,6 +436,11 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
|
||||
else if (new_indices)
|
||||
create.columns_list->set(create.columns_list->indices, new_indices);
|
||||
|
||||
if (new_constraints && create.columns_list->constraints)
|
||||
create.columns_list->replace(create.columns_list->constraints, new_constraints);
|
||||
else if (new_constraints)
|
||||
create.columns_list->set(create.columns_list->constraints, new_constraints);
|
||||
|
||||
/// Check for duplicates
|
||||
std::set<String> all_columns;
|
||||
for (const auto & column : columns)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
|
||||
@ -30,6 +31,7 @@ public:
|
||||
static ASTPtr formatColumns(const ColumnsDescription & columns);
|
||||
|
||||
static ASTPtr formatIndices(const IndicesDescription & indices);
|
||||
static ASTPtr formatConstraints(const ConstraintsDescription & constraints);
|
||||
|
||||
void setForceRestoreData(bool has_force_restore_data_flag_)
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
|
||||
#include <DataStreams/AddingDefaultBlockOutputStream.h>
|
||||
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
||||
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
|
||||
#include <DataStreams/OwningBlockInputStream.h>
|
||||
#include <DataStreams/ConvertingBlockInputStream.h>
|
||||
#include <DataStreams/CountingBlockOutputStream.h>
|
||||
@ -120,6 +121,9 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
out = std::make_shared<AddingDefaultBlockOutputStream>(
|
||||
out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context);
|
||||
|
||||
out = std::make_shared<CheckConstraintsBlockOutputStream>(
|
||||
query.table, out, query_sample_block, table->getConstraints(), context);
|
||||
|
||||
auto out_wrapper = std::make_shared<CountingBlockOutputStream>(out);
|
||||
out_wrapper->setProcessListElement(context.getProcessListElement());
|
||||
out = std::move(out_wrapper);
|
||||
|
@ -1550,7 +1550,11 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
{
|
||||
/// Unify streams in case they have different headers.
|
||||
auto first_header = streams.at(0)->getHeader();
|
||||
for (size_t i = 1; i < streams.size(); ++i)
|
||||
|
||||
if (first_header.columns() > 1 && first_header.has("_dummy"))
|
||||
first_header.erase("_dummy");
|
||||
|
||||
for (size_t i = 0; i < streams.size(); ++i)
|
||||
{
|
||||
auto & stream = streams[i];
|
||||
auto header = stream->getHeader();
|
||||
|
@ -25,6 +25,7 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int TOO_MANY_COLUMNS;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr InterpreterWatchQuery::executeImpl()
|
||||
@ -34,6 +35,9 @@ BlockInputStreamPtr InterpreterWatchQuery::executeImpl()
|
||||
|
||||
BlockIO InterpreterWatchQuery::execute()
|
||||
{
|
||||
if (!context.getSettingsRef().allow_experimental_live_view)
|
||||
throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
|
||||
BlockIO res;
|
||||
const ASTWatchQuery & query = typeid_cast<const ASTWatchQuery &>(*query_ptr);
|
||||
String database;
|
||||
|
@ -133,12 +133,29 @@ void ASTAlterCommand::formatImpl(
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::ADD_CONSTRAINT)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD CONSTRAINT" << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
constraint_decl->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_CONSTRAINT)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
constraint->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (detach ? "DETACH" : "DROP") << " PARTITION "
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_DETACHED_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP DETACHED" << (part ? " PART " : " PARTITION ")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::ATTACH_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ATTACH "
|
||||
@ -209,42 +226,6 @@ void ASTAlterCommand::formatImpl(
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_ADD)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_DROP)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_MODIFY)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_SUSPEND)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "SUSPEND " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_RESUME)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RESUME " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::LIVE_CHANNEL_REFRESH)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
values->formatImpl(settings, state, frame);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
@ -299,8 +280,6 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
|
||||
|
||||
if (is_live_view)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : "");
|
||||
else if (is_live_channel)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE CHANNEL " << (settings.hilite ? hilite_none : "");
|
||||
else
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
|
@ -17,13 +17,6 @@ namespace DB
|
||||
* COMMENT_COLUMN col_name 'comment',
|
||||
* ALTER LIVE VIEW [db.]name_type
|
||||
* REFRESH
|
||||
* ALTER CHANNEL [db.]name_type
|
||||
* ADD live_view,...
|
||||
* DROP live_view,...
|
||||
* SUSPEND live_view,...
|
||||
* RESUME live_view,...
|
||||
* REFRESH live_view,...
|
||||
* MODIFY live_view,...
|
||||
*/
|
||||
|
||||
class ASTAlterCommand : public IAST
|
||||
@ -42,7 +35,11 @@ public:
|
||||
DROP_INDEX,
|
||||
MATERIALIZE_INDEX,
|
||||
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
|
||||
DROP_PARTITION,
|
||||
DROP_DETACHED_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
REPLACE_PARTITION,
|
||||
FETCH_PARTITION,
|
||||
@ -55,13 +52,6 @@ public:
|
||||
NO_TYPE,
|
||||
|
||||
LIVE_VIEW_REFRESH,
|
||||
|
||||
LIVE_CHANNEL_ADD,
|
||||
LIVE_CHANNEL_DROP,
|
||||
LIVE_CHANNEL_SUSPEND,
|
||||
LIVE_CHANNEL_RESUME,
|
||||
LIVE_CHANNEL_REFRESH,
|
||||
LIVE_CHANNEL_MODIFY
|
||||
};
|
||||
|
||||
Type type = NO_TYPE;
|
||||
@ -92,6 +82,14 @@ public:
|
||||
*/
|
||||
ASTPtr index;
|
||||
|
||||
/** The ADD CONSTRAINT query stores the ConstraintDeclaration there.
|
||||
*/
|
||||
ASTPtr constraint_decl;
|
||||
|
||||
/** The DROP CONSTRAINT query stores the name for deletion.
|
||||
*/
|
||||
ASTPtr constraint;
|
||||
|
||||
/** Used in DROP PARTITION and ATTACH PARTITION FROM queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
*/
|
||||
@ -115,7 +113,7 @@ public:
|
||||
|
||||
bool detach = false; /// true for DETACH PARTITION
|
||||
|
||||
bool part = false; /// true for ATTACH PART
|
||||
bool part = false; /// true for ATTACH PART and DROP DETACHED PART
|
||||
|
||||
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
|
||||
|
||||
@ -170,7 +168,6 @@ class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCl
|
||||
{
|
||||
public:
|
||||
bool is_live_view{false}; /// true for ALTER LIVE VIEW
|
||||
bool is_live_channel{false}; /// true for ALTER LIVE CHANNEL
|
||||
|
||||
ASTAlterCommandList * command_list = nullptr;
|
||||
|
||||
|
29
dbms/src/Parsers/ASTConstraintDeclaration.cpp
Normal file
29
dbms/src/Parsers/ASTConstraintDeclaration.cpp
Normal file
@ -0,0 +1,29 @@
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTConstraintDeclaration::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTConstraintDeclaration>();
|
||||
|
||||
res->name = name;
|
||||
|
||||
if (expr)
|
||||
res->set(res->expr, expr->clone());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
|
||||
s.ostr << s.nl_or_ws << indent_str;
|
||||
s.ostr << backQuoteIfNeed(name);
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : "");
|
||||
expr->formatImpl(s, state, frame);
|
||||
}
|
||||
|
||||
}
|
22
dbms/src/Parsers/ASTConstraintDeclaration.h
Normal file
22
dbms/src/Parsers/ASTConstraintDeclaration.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** name CHECK logical_expr
|
||||
*/
|
||||
class ASTConstraintDeclaration : public IAST
|
||||
{
|
||||
public:
|
||||
String name;
|
||||
IAST * expr;
|
||||
|
||||
String getID(char) const override { return "Constraint"; }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
}
|
@ -128,6 +128,8 @@ ASTPtr ASTColumns::clone() const
|
||||
res->set(res->columns, columns->clone());
|
||||
if (indices)
|
||||
res->set(res->indices, indices->clone());
|
||||
if (constraints)
|
||||
res->set(res->constraints, constraints->clone());
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -156,6 +158,16 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma
|
||||
list.children.push_back(elem);
|
||||
}
|
||||
}
|
||||
if (constraints)
|
||||
{
|
||||
for (const auto & constraint : constraints->children)
|
||||
{
|
||||
auto elem = std::make_shared<ASTColumnsElement>();
|
||||
elem->prefix = "CONSTRAINT";
|
||||
elem->set(elem->elem, constraint->clone());
|
||||
list.children.push_back(elem);
|
||||
}
|
||||
}
|
||||
|
||||
if (!list.children.empty())
|
||||
list.formatImpl(s, state, frame);
|
||||
@ -208,9 +220,6 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
what = "MATERIALIZED VIEW";
|
||||
if (is_live_view)
|
||||
what = "LIVE VIEW";
|
||||
if (is_live_channel)
|
||||
what = "LIVE CHANNEL";
|
||||
|
||||
|
||||
settings.ostr
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
|
@ -36,6 +36,7 @@ class ASTColumns : public IAST
|
||||
public:
|
||||
ASTExpressionList * columns = nullptr;
|
||||
ASTExpressionList * indices = nullptr;
|
||||
ASTExpressionList * constraints = nullptr;
|
||||
|
||||
String getID(char) const override { return "Columns definition"; }
|
||||
|
||||
@ -56,7 +57,6 @@ public:
|
||||
bool is_view{false};
|
||||
bool is_materialized_view{false};
|
||||
bool is_live_view{false};
|
||||
bool is_live_channel{false};
|
||||
bool is_populate{false};
|
||||
bool replace_view{false}; /// CREATE OR REPLACE VIEW
|
||||
ASTColumns * columns_list = nullptr;
|
||||
|
@ -34,6 +34,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_clear_index("CLEAR INDEX");
|
||||
ParserKeyword s_materialize_index("MATERIALIZE INDEX");
|
||||
|
||||
ParserKeyword s_add_constraint("ADD CONSTRAINT");
|
||||
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
|
||||
|
||||
ParserKeyword s_add("ADD");
|
||||
ParserKeyword s_drop("DROP");
|
||||
ParserKeyword s_suspend("SUSPEND");
|
||||
@ -44,6 +47,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
||||
ParserKeyword s_detach_partition("DETACH PARTITION");
|
||||
ParserKeyword s_drop_partition("DROP PARTITION");
|
||||
ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION");
|
||||
ParserKeyword s_drop_detached_part("DROP DETACHED PART");
|
||||
ParserKeyword s_attach_part("ATTACH PART");
|
||||
ParserKeyword s_fetch_partition("FETCH PARTITION");
|
||||
ParserKeyword s_replace_partition("REPLACE PARTITION");
|
||||
@ -66,6 +71,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserStringLiteral parser_string_literal;
|
||||
ParserCompoundColumnDeclaration parser_col_decl;
|
||||
ParserIndexDeclaration parser_idx_decl;
|
||||
ParserConstraintDeclaration parser_constraint_decl;
|
||||
ParserCompoundColumnDeclaration parser_modify_col_decl(false);
|
||||
ParserPartition parser_partition;
|
||||
ParserExpression parser_exp_elem;
|
||||
@ -83,53 +89,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (is_live_channel)
|
||||
{
|
||||
if (s_add.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_ADD;
|
||||
}
|
||||
else if (s_drop.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_DROP;
|
||||
}
|
||||
else if (s_suspend.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_SUSPEND;
|
||||
}
|
||||
else if (s_resume.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_RESUME;
|
||||
}
|
||||
else if (s_refresh.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_REFRESH;
|
||||
}
|
||||
else if (s_modify.ignore(pos, expected))
|
||||
{
|
||||
if (!values_p.parse(pos, command->values, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::LIVE_CHANNEL_MODIFY;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (s_add_column.ignore(pos, expected))
|
||||
@ -155,6 +114,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PARTITION;
|
||||
}
|
||||
else if (s_drop_detached_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_DETACHED_PARTITION;
|
||||
}
|
||||
else if (s_drop_detached_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_DETACHED_PARTITION;
|
||||
command->part = true;
|
||||
}
|
||||
else if (s_drop_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
@ -163,8 +137,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
command->detach = false;
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
{
|
||||
@ -245,6 +219,27 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_add_constraint.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_constraint_decl.parse(pos, command->constraint_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::ADD_CONSTRAINT;
|
||||
}
|
||||
else if (s_drop_constraint.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->constraint, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_CONSTRAINT;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_detach_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
@ -424,7 +419,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
node = command_list;
|
||||
|
||||
ParserToken s_comma(TokenType::Comma);
|
||||
ParserAlterCommand p_command(is_live_view, is_live_channel);
|
||||
ParserAlterCommand p_command(is_live_view);
|
||||
|
||||
do
|
||||
{
|
||||
@ -474,20 +469,13 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
ParserKeyword s_alter_table("ALTER TABLE");
|
||||
ParserKeyword s_alter_live_view("ALTER LIVE VIEW");
|
||||
ParserKeyword s_alter_live_channel("ALTER LIVE CHANNEL");
|
||||
|
||||
bool is_live_view = false;
|
||||
bool is_live_channel = false;
|
||||
|
||||
if (!s_alter_table.ignore(pos, expected))
|
||||
{
|
||||
if (!s_alter_live_view.ignore(pos, expected))
|
||||
{
|
||||
if (!s_alter_live_channel.ignore(pos, expected))
|
||||
return false;
|
||||
else
|
||||
is_live_channel = true;
|
||||
}
|
||||
return false;
|
||||
else
|
||||
is_live_view = true;
|
||||
}
|
||||
@ -495,9 +483,6 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (is_live_view)
|
||||
query->is_live_view = true;
|
||||
|
||||
if (is_live_channel)
|
||||
query->is_live_channel = true;
|
||||
|
||||
if (!parseDatabaseAndTableName(pos, expected, query->database, query->table))
|
||||
return false;
|
||||
|
||||
@ -509,7 +494,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
query->cluster = cluster_str;
|
||||
|
||||
ParserAlterCommandList p_command_list(is_live_view, is_live_channel);
|
||||
ParserAlterCommandList p_command_list(is_live_view);
|
||||
ASTPtr command_list;
|
||||
if (!p_command_list.parse(pos, command_list, expected))
|
||||
return false;
|
||||
|
@ -21,13 +21,6 @@ namespace DB
|
||||
* [UPDATE col_name = expr, ... WHERE ...]
|
||||
* ALTER LIVE VIEW [db.name]
|
||||
* [REFRESH]
|
||||
* ALTER LIVE CHANNEL [db.name] [ON CLUSTER cluster]
|
||||
* [ADD live_view, ...]
|
||||
* [DROP live_view, ...]
|
||||
* [SUSPEND live_view, ...]
|
||||
* [RESUME live_view, ...]
|
||||
* [REFRESH live_view, ...]
|
||||
* [MODIFY live_view, ...]
|
||||
*/
|
||||
|
||||
class ParserAlterQuery : public IParserBase
|
||||
@ -46,9 +39,8 @@ protected:
|
||||
|
||||
public:
|
||||
bool is_live_view;
|
||||
bool is_live_channel;
|
||||
|
||||
ParserAlterCommandList(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {}
|
||||
ParserAlterCommandList(bool is_live_view_ = false) : is_live_view(is_live_view_) {}
|
||||
};
|
||||
|
||||
|
||||
@ -60,9 +52,8 @@ protected:
|
||||
|
||||
public:
|
||||
bool is_live_view;
|
||||
bool is_live_channel;
|
||||
|
||||
ParserAlterCommand(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {}
|
||||
ParserAlterCommand(bool is_live_view_ = false) : is_live_view(is_live_view_) {}
|
||||
};
|
||||
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserSelectWithUnionQuery.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -143,12 +144,41 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_check("CHECK");
|
||||
|
||||
bool ParserColumnAndIndexDeclaraion::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserIdentifier name_p;
|
||||
ParserLogicalOrExpression expression_p;
|
||||
|
||||
ASTPtr name;
|
||||
ASTPtr expr;
|
||||
|
||||
if (!name_p.parse(pos, name, expected))
|
||||
return false;
|
||||
|
||||
if (!s_check.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!expression_p.parse(pos, expr, expected))
|
||||
return false;
|
||||
|
||||
auto constraint = std::make_shared<ASTConstraintDeclaration>();
|
||||
constraint->name = name->as<ASTIdentifier &>().name;
|
||||
constraint->set(constraint->expr, expr);
|
||||
node = constraint;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_constraint("CONSTRAINT");
|
||||
|
||||
ParserIndexDeclaration index_p;
|
||||
ParserConstraintDeclaration constraint_p;
|
||||
ParserColumnDeclaration column_p;
|
||||
|
||||
ASTPtr new_node = nullptr;
|
||||
@ -158,6 +188,11 @@ bool ParserColumnAndIndexDeclaraion::parseImpl(Pos & pos, ASTPtr & node, Expecte
|
||||
if (!index_p.parse(pos, new_node, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_constraint.ignore(pos, expected))
|
||||
{
|
||||
if (!constraint_p.parse(pos, new_node, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!column_p.parse(pos, new_node, expected))
|
||||
@ -174,16 +209,24 @@ bool ParserIndexDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool ParserConstraintDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
return ParserList(std::make_unique<ParserConstraintDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||
.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr list;
|
||||
if (!ParserList(std::make_unique<ParserColumnAndIndexDeclaraion>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||
if (!ParserList(
|
||||
std::make_unique<ParserTablePropertyDeclaration>(),
|
||||
std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||
.parse(pos, list, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr columns = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr indices = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr constraints = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & elem : list->children)
|
||||
{
|
||||
@ -191,6 +234,8 @@ bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node,
|
||||
columns->children.push_back(elem);
|
||||
else if (elem->as<ASTIndexDeclaration>())
|
||||
indices->children.push_back(elem);
|
||||
else if (elem->as<ASTConstraintDeclaration>())
|
||||
constraints->children.push_back(elem);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
@ -201,6 +246,8 @@ bool ParserColumnsOrIndicesDeclarationList::parseImpl(Pos & pos, ASTPtr & node,
|
||||
res->set(res->columns, columns);
|
||||
if (!indices->children.empty())
|
||||
res->set(res->indices, indices);
|
||||
if (!constraints->children.empty())
|
||||
res->set(res->constraints, constraints);
|
||||
|
||||
node = res;
|
||||
|
||||
@ -318,7 +365,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword s_with("WITH");
|
||||
ParserKeyword s_materialized("MATERIALIZED");
|
||||
ParserKeyword s_live("LIVE");
|
||||
ParserKeyword s_channel("CHANNEL");
|
||||
ParserKeyword s_populate("POPULATE");
|
||||
ParserKeyword s_or_replace("OR REPLACE");
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
@ -326,7 +372,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||
ParserStorage storage_p;
|
||||
ParserIdentifier name_p;
|
||||
ParserColumnsOrIndicesDeclarationList columns_or_indices_p;
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserFunction table_function_p;
|
||||
ParserNameList names_p;
|
||||
@ -349,7 +395,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool is_view = false;
|
||||
bool is_materialized_view = false;
|
||||
bool is_live_view = false;
|
||||
bool is_live_channel = false;
|
||||
bool is_populate = false;
|
||||
bool is_temporary = false;
|
||||
bool replace_view = false;
|
||||
@ -407,7 +452,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
/// List of columns.
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
if (!columns_or_indices_p.parse(pos, columns_list, expected))
|
||||
if (!table_properties_p.parse(pos, columns_list, expected))
|
||||
return false;
|
||||
|
||||
if (!s_rparen.ignore(pos, expected))
|
||||
@ -447,9 +492,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
else if (s_live.ignore(pos, expected))
|
||||
{
|
||||
if (s_channel.ignore(pos, expected))
|
||||
is_live_channel = true;
|
||||
else if (s_view.ignore(pos, expected))
|
||||
if (s_view.ignore(pos, expected))
|
||||
is_live_view = true;
|
||||
else
|
||||
return false;
|
||||
@ -473,50 +516,36 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!is_live_channel)
|
||||
// TO [db.]table
|
||||
if (ParserKeyword{"TO"}.ignore(pos, expected))
|
||||
{
|
||||
// TO [db.]table
|
||||
if (ParserKeyword{"TO"}.ignore(pos, expected))
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
to_database = to_table;
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
to_database = to_table;
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
if (!columns_or_indices_p.parse(pos, columns_list, expected))
|
||||
if (!table_properties_p.parse(pos, columns_list, expected))
|
||||
return false;
|
||||
|
||||
if (!s_rparen.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (is_live_channel)
|
||||
{
|
||||
if (s_with.ignore(pos, expected))
|
||||
{
|
||||
if (!names_p.parse(pos, tables, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// AS SELECT ...
|
||||
if (!s_as.ignore(pos, expected))
|
||||
return false;
|
||||
/// AS SELECT ...
|
||||
if (!s_as.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!select_p.parse(pos, select, expected))
|
||||
return false;
|
||||
}
|
||||
if (!select_p.parse(pos, select, expected))
|
||||
return false;
|
||||
}
|
||||
else if (is_temporary)
|
||||
return false;
|
||||
@ -590,7 +619,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
if (!columns_or_indices_p.parse(pos, columns_list, expected))
|
||||
if (!table_properties_p.parse(pos, columns_list, expected))
|
||||
return false;
|
||||
|
||||
if (!s_rparen.ignore(pos, expected))
|
||||
@ -626,7 +655,6 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
query->is_view = is_view;
|
||||
query->is_materialized_view = is_materialized_view;
|
||||
query->is_live_view = is_live_view;
|
||||
query->is_live_channel = is_live_channel;
|
||||
query->is_populate = is_populate;
|
||||
query->temporary = is_temporary;
|
||||
query->replace_view = replace_view;
|
||||
|
@ -252,11 +252,17 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserColumnAndIndexDeclaraion : public IParserBase
|
||||
class ParserConstraintDeclaration : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "column or index declaration"; }
|
||||
const char * getName() const override { return "constraint declaration"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserTablePropertyDeclaration : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "table propery (column, index, constraint) declaration"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
@ -268,8 +274,15 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserConstraintDeclarationList : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "constraint declaration list"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserColumnsOrIndicesDeclarationList : public IParserBase
|
||||
|
||||
class ParserTablePropertiesDeclarationList : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "columns or indices declaration list"; }
|
||||
@ -308,7 +321,7 @@ protected:
|
||||
* CREATE|ATTACH DATABASE db [ENGINE = engine]
|
||||
*
|
||||
* Or:
|
||||
* CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [[TEMPORARY] LIVE [CHANNEL] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
* CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
*/
|
||||
class ParserCreateQuery : public IParserBase
|
||||
{
|
||||
|
@ -590,32 +590,45 @@ void PipelineExecutor::executeImpl(size_t num_threads)
|
||||
for (size_t i = 0; i < num_threads; ++i)
|
||||
executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
|
||||
|
||||
auto thread_group = CurrentThread::getGroup();
|
||||
|
||||
using ThreadsData = std::vector<ThreadFromGlobalPool>;
|
||||
ThreadsData threads;
|
||||
threads.reserve(num_threads);
|
||||
|
||||
bool finished_flag = false;
|
||||
|
||||
SCOPE_EXIT(
|
||||
if (!finished_flag)
|
||||
{
|
||||
finish();
|
||||
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
}
|
||||
);
|
||||
|
||||
addChildlessProcessorsToStack(stack);
|
||||
|
||||
while (!stack.empty())
|
||||
{
|
||||
UInt64 proc = stack.top();
|
||||
stack.pop();
|
||||
std::lock_guard lock(task_queue_mutex);
|
||||
|
||||
if (prepareProcessor(proc, stack, stack, 0, false))
|
||||
while (!stack.empty())
|
||||
{
|
||||
auto cur_state = graph[proc].execution_state.get();
|
||||
task_queue.push(cur_state);
|
||||
UInt64 proc = stack.top();
|
||||
stack.pop();
|
||||
|
||||
if (prepareProcessor(proc, stack, stack, 0, false))
|
||||
{
|
||||
auto cur_state = graph[proc].execution_state.get();
|
||||
task_queue.push(cur_state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ThreadPool pool(num_threads);
|
||||
|
||||
SCOPE_EXIT(
|
||||
finish();
|
||||
pool.wait()
|
||||
);
|
||||
|
||||
auto thread_group = CurrentThread::getGroup();
|
||||
|
||||
for (size_t i = 0; i < num_threads; ++i)
|
||||
{
|
||||
pool.schedule([this, thread_group, thread_num = i, num_threads]
|
||||
threads.emplace_back([this, thread_group, thread_num = i, num_threads]
|
||||
{
|
||||
/// ThreadStatus thread_status;
|
||||
|
||||
@ -631,7 +644,10 @@ void PipelineExecutor::executeImpl(size_t num_threads)
|
||||
});
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
finished_flag = true;
|
||||
}
|
||||
|
||||
String PipelineExecutor::dumpPipeline() const
|
||||
|
@ -5,7 +5,7 @@ namespace DB
|
||||
{
|
||||
|
||||
CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_)
|
||||
: IInflatingTransform(std::move(header), params_->getHeader())
|
||||
: IAccumulatingTransform(std::move(header), params_->getHeader())
|
||||
, params(std::move(params_))
|
||||
, keys(params->params.keys)
|
||||
{
|
||||
@ -13,28 +13,45 @@ CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_
|
||||
throw Exception("Too many keys are used for CubeTransform.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void CubeTransform::consume(Chunk chunk)
|
||||
Chunk CubeTransform::merge(Chunks && chunks, bool final)
|
||||
{
|
||||
consumed_chunk = std::move(chunk);
|
||||
auto num_rows = consumed_chunk.getNumRows();
|
||||
mask = (UInt64(1) << keys.size()) - 1;
|
||||
BlocksList rollup_blocks;
|
||||
for (auto & chunk : chunks)
|
||||
rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()));
|
||||
|
||||
current_columns = consumed_chunk.getColumns();
|
||||
current_zero_columns.clear();
|
||||
current_zero_columns.reserve(keys.size());
|
||||
|
||||
for (auto key : keys)
|
||||
current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows));
|
||||
auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final);
|
||||
auto num_rows = rollup_block.rows();
|
||||
return Chunk(rollup_block.getColumns(), num_rows);
|
||||
}
|
||||
|
||||
bool CubeTransform::canGenerate()
|
||||
void CubeTransform::consume(Chunk chunk)
|
||||
{
|
||||
return consumed_chunk;
|
||||
consumed_chunks.emplace_back(std::move(chunk));
|
||||
}
|
||||
|
||||
Chunk CubeTransform::generate()
|
||||
{
|
||||
auto gen_chunk = std::move(consumed_chunk);
|
||||
if (!consumed_chunks.empty())
|
||||
{
|
||||
if (consumed_chunks.size() > 1)
|
||||
cube_chunk = merge(std::move(consumed_chunks), false);
|
||||
else
|
||||
cube_chunk = std::move(consumed_chunks.front());
|
||||
|
||||
consumed_chunks.clear();
|
||||
|
||||
auto num_rows = cube_chunk.getNumRows();
|
||||
mask = (UInt64(1) << keys.size()) - 1;
|
||||
|
||||
current_columns = cube_chunk.getColumns();
|
||||
current_zero_columns.clear();
|
||||
current_zero_columns.reserve(keys.size());
|
||||
|
||||
for (auto key : keys)
|
||||
current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows));
|
||||
}
|
||||
|
||||
auto gen_chunk = std::move(cube_chunk);
|
||||
|
||||
if (mask)
|
||||
{
|
||||
@ -47,11 +64,9 @@ Chunk CubeTransform::generate()
|
||||
if ((mask & (UInt64(1) << (size - i - 1))) == 0)
|
||||
columns[keys[i]] = current_zero_columns[i];
|
||||
|
||||
BlocksList cube_blocks = { getInputPort().getHeader().cloneWithColumns(columns) };
|
||||
auto cube_block = params->aggregator.mergeBlocks(cube_blocks, false);
|
||||
|
||||
auto num_rows = cube_block.rows();
|
||||
consumed_chunk = Chunk(cube_block.getColumns(), num_rows);
|
||||
Chunks chunks;
|
||||
chunks.emplace_back(std::move(columns), current_columns.front()->size());
|
||||
cube_chunk = merge(std::move(chunks), false);
|
||||
}
|
||||
|
||||
finalizeChunk(gen_chunk);
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
|
||||
/// Takes blocks after grouping, with non-finalized aggregate functions.
|
||||
/// Calculates all subsets of columns and aggregates over them.
|
||||
class CubeTransform : public IInflatingTransform
|
||||
class CubeTransform : public IAccumulatingTransform
|
||||
{
|
||||
public:
|
||||
CubeTransform(Block header, AggregatingTransformParamsPtr params);
|
||||
@ -16,20 +16,20 @@ public:
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override;
|
||||
|
||||
bool canGenerate() override;
|
||||
|
||||
Chunk generate() override;
|
||||
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
ColumnNumbers keys;
|
||||
|
||||
Chunk consumed_chunk;
|
||||
Chunks consumed_chunks;
|
||||
Chunk cube_chunk;
|
||||
Columns current_columns;
|
||||
Columns current_zero_columns;
|
||||
|
||||
UInt64 mask = 0;
|
||||
|
||||
Chunk merge(Chunks && chunks, bool final);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,7 @@ static Block transformHeader(Block header, const ExpressionActionsPtr & expressi
|
||||
|
||||
|
||||
ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_, bool on_totals_, bool default_totals_)
|
||||
: ISimpleTransform(header_, transformHeader(header_, expression), on_totals_)
|
||||
: ISimpleTransform(header_, transformHeader(header_, expression_), on_totals_)
|
||||
, expression(std::move(expression_))
|
||||
, on_totals(on_totals_)
|
||||
, default_totals(default_totals_)
|
||||
|
@ -334,7 +334,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk)
|
||||
|
||||
|
||||
SortingAggregatedTransform::SortingAggregatedTransform(size_t num_inputs_, AggregatingTransformParamsPtr params_)
|
||||
: IProcessor(InputPorts(num_inputs_, params->getHeader()), {params_->getHeader()})
|
||||
: IProcessor(InputPorts(num_inputs_, params_->getHeader()), {params_->getHeader()})
|
||||
, num_inputs(num_inputs_)
|
||||
, params(std::move(params_))
|
||||
, last_bucket_number(num_inputs, -1)
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -13,7 +14,7 @@ MergingSortedTransform::MergingSortedTransform(
|
||||
UInt64 limit_,
|
||||
bool quiet_,
|
||||
bool have_all_inputs_)
|
||||
: IProcessor(InputPorts(num_inputs, header), {header})
|
||||
: IProcessor(InputPorts(num_inputs, header), {materializeBlock(header)})
|
||||
, description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, have_all_inputs(have_all_inputs_)
|
||||
, merged_data(header), source_chunks(num_inputs), cursors(num_inputs)
|
||||
|
@ -93,7 +93,7 @@ protected:
|
||||
columns = chunk.mutateColumns();
|
||||
if (limit_rows && num_rows > limit_rows)
|
||||
for (auto & column : columns)
|
||||
column = (*column->cut(0, limit_rows)).mutate();
|
||||
column = (*column->cut(0, limit_rows)->convertToFullColumnIfConst()).mutate();
|
||||
|
||||
total_merged_rows += num_rows;
|
||||
merged_rows = num_rows;
|
||||
@ -165,6 +165,13 @@ private:
|
||||
|
||||
void updateCursor(Chunk chunk, size_t source_num)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
|
||||
auto & shared_chunk_ptr = source_chunks[source_num];
|
||||
|
||||
if (!shared_chunk_ptr)
|
||||
|
@ -5,7 +5,7 @@ namespace DB
|
||||
{
|
||||
|
||||
RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_)
|
||||
: IInflatingTransform(std::move(header), params_->getHeader())
|
||||
: IAccumulatingTransform(std::move(header), params_->getHeader())
|
||||
, params(std::move(params_))
|
||||
, keys(params->params.keys)
|
||||
{
|
||||
@ -13,18 +13,34 @@ RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr par
|
||||
|
||||
void RollupTransform::consume(Chunk chunk)
|
||||
{
|
||||
consumed_chunk = std::move(chunk);
|
||||
last_removed_key = keys.size();
|
||||
consumed_chunks.emplace_back(std::move(chunk));
|
||||
}
|
||||
|
||||
bool RollupTransform::canGenerate()
|
||||
Chunk RollupTransform::merge(Chunks && chunks, bool final)
|
||||
{
|
||||
return consumed_chunk;
|
||||
BlocksList rollup_blocks;
|
||||
for (auto & chunk : chunks)
|
||||
rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()));
|
||||
|
||||
auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final);
|
||||
auto num_rows = rollup_block.rows();
|
||||
return Chunk(rollup_block.getColumns(), num_rows);
|
||||
}
|
||||
|
||||
Chunk RollupTransform::generate()
|
||||
{
|
||||
auto gen_chunk = std::move(consumed_chunk);
|
||||
if (!consumed_chunks.empty())
|
||||
{
|
||||
if (consumed_chunks.size() > 1)
|
||||
rollup_chunk = merge(std::move(consumed_chunks), false);
|
||||
else
|
||||
rollup_chunk = std::move(consumed_chunks.front());
|
||||
|
||||
consumed_chunks.clear();
|
||||
last_removed_key = keys.size();
|
||||
}
|
||||
|
||||
auto gen_chunk = std::move(rollup_chunk);
|
||||
|
||||
if (last_removed_key)
|
||||
{
|
||||
@ -35,11 +51,9 @@ Chunk RollupTransform::generate()
|
||||
auto columns = gen_chunk.getColumns();
|
||||
columns[key] = columns[key]->cloneEmpty()->cloneResized(num_rows);
|
||||
|
||||
BlocksList rollup_blocks = { getInputPort().getHeader().cloneWithColumns(columns) };
|
||||
auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, false);
|
||||
|
||||
num_rows = rollup_block.rows();
|
||||
consumed_chunk = Chunk(rollup_block.getColumns(), num_rows);
|
||||
Chunks chunks;
|
||||
chunks.emplace_back(std::move(columns), num_rows);
|
||||
rollup_chunk = merge(std::move(chunks), false);
|
||||
}
|
||||
|
||||
finalizeChunk(gen_chunk);
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Processors/IInflatingTransform.h>
|
||||
#include <Processors/IAccumulatingTransform.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
|
||||
namespace DB
|
||||
@ -7,7 +7,7 @@ namespace DB
|
||||
|
||||
/// Takes blocks after grouping, with non-finalized aggregate functions.
|
||||
/// Calculates subtotals and grand totals values for a set of columns.
|
||||
class RollupTransform : public IInflatingTransform
|
||||
class RollupTransform : public IAccumulatingTransform
|
||||
{
|
||||
public:
|
||||
RollupTransform(Block header, AggregatingTransformParamsPtr params);
|
||||
@ -15,14 +15,16 @@ public:
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override;
|
||||
bool canGenerate() override;
|
||||
Chunk generate() override;
|
||||
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
ColumnNumbers keys;
|
||||
Chunk consumed_chunk;
|
||||
Chunks consumed_chunks;
|
||||
Chunk rollup_chunk;
|
||||
size_t last_removed_key = 0;
|
||||
|
||||
Chunk merge(Chunks && chunks, bool final);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -161,6 +161,8 @@ void TotalsHavingTransform::transform(Chunk & chunk)
|
||||
if (const_filter_description.always_true)
|
||||
{
|
||||
addToTotals(chunk, nullptr);
|
||||
auto num_rows = columns.front()->size();
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -153,6 +154,32 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.constraint_decl = command_ast->constraint_decl;
|
||||
command.type = AlterCommand::ADD_CONSTRAINT;
|
||||
|
||||
const auto & ast_constraint_decl = command_ast->constraint_decl->as<ASTConstraintDeclaration &>();
|
||||
|
||||
command.constraint_name = ast_constraint_decl.name;
|
||||
|
||||
command.if_not_exists = command_ast->if_not_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT && !command_ast->partition)
|
||||
{
|
||||
if (command_ast->clear_column)
|
||||
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
AlterCommand command;
|
||||
command.if_exists = command_ast->if_exists;
|
||||
command.type = AlterCommand::DROP_CONSTRAINT;
|
||||
command.constraint_name = command_ast->constraint->as<ASTIdentifier &>().name;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_INDEX && !command_ast->partition)
|
||||
{
|
||||
if (command_ast->clear_column)
|
||||
@ -178,7 +205,8 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
|
||||
void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
{
|
||||
if (type == ADD_COLUMN)
|
||||
{
|
||||
@ -302,6 +330,45 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
|
||||
|
||||
indices_description.indices.erase(erase_it);
|
||||
}
|
||||
else if (type == ADD_CONSTRAINT)
|
||||
{
|
||||
if (std::any_of(
|
||||
constraints_description.constraints.cbegin(),
|
||||
constraints_description.constraints.cend(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
}))
|
||||
{
|
||||
if (if_not_exists)
|
||||
return;
|
||||
throw Exception("Cannot add constraint " + constraint_name + ": constraint with this name already exists",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
auto insert_it = constraints_description.constraints.end();
|
||||
|
||||
constraints_description.constraints.emplace(insert_it, std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint_decl));
|
||||
}
|
||||
else if (type == DROP_CONSTRAINT)
|
||||
{
|
||||
auto erase_it = std::find_if(
|
||||
constraints_description.constraints.begin(),
|
||||
constraints_description.constraints.end(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
});
|
||||
|
||||
if (erase_it == constraints_description.constraints.end())
|
||||
{
|
||||
if (if_exists)
|
||||
return;
|
||||
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
constraints_description.constraints.erase(erase_it);
|
||||
}
|
||||
else if (type == MODIFY_TTL)
|
||||
{
|
||||
ttl_table_ast = ttl;
|
||||
@ -321,20 +388,23 @@ bool AlterCommand::isMutable() const
|
||||
}
|
||||
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
{
|
||||
auto new_columns_description = columns_description;
|
||||
auto new_indices_description = indices_description;
|
||||
auto new_constraints_description = constraints_description;
|
||||
auto new_order_by_ast = order_by_ast;
|
||||
auto new_primary_key_ast = primary_key_ast;
|
||||
auto new_ttl_table_ast = ttl_table_ast;
|
||||
|
||||
for (const AlterCommand & command : *this)
|
||||
if (!command.ignore)
|
||||
command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
command.apply(new_columns_description, new_indices_description, new_constraints_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
columns_description = std::move(new_columns_description);
|
||||
indices_description = std::move(new_indices_description);
|
||||
constraints_description = std::move(new_constraints_description);
|
||||
order_by_ast = std::move(new_order_by_ast);
|
||||
primary_key_ast = std::move(new_primary_key_ast);
|
||||
ttl_table_ast = std::move(new_ttl_table_ast);
|
||||
@ -522,10 +592,11 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
{
|
||||
auto out_columns_description = columns_description;
|
||||
IndicesDescription indices_description;
|
||||
ConstraintsDescription constraints_description;
|
||||
ASTPtr out_order_by;
|
||||
ASTPtr out_primary_key;
|
||||
ASTPtr out_ttl_table;
|
||||
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table);
|
||||
apply(out_columns_description, indices_description, constraints_description, out_order_by, out_primary_key, out_ttl_table);
|
||||
|
||||
if (out_order_by)
|
||||
throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -533,6 +604,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (!indices_description.indices.empty())
|
||||
throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (!constraints_description.constraints.empty())
|
||||
throw Exception("Storage doesn't support modifying constraints", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (out_ttl_table)
|
||||
throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,6 +26,8 @@ struct AlterCommand
|
||||
MODIFY_ORDER_BY,
|
||||
ADD_INDEX,
|
||||
DROP_INDEX,
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
MODIFY_TTL,
|
||||
UKNOWN_TYPE,
|
||||
};
|
||||
@ -62,6 +65,12 @@ struct AlterCommand
|
||||
/// For ADD/DROP INDEX
|
||||
String index_name;
|
||||
|
||||
// For ADD CONSTRAINT
|
||||
ASTPtr constraint_decl;
|
||||
|
||||
// For ADD/DROP CONSTRAINT
|
||||
String constraint_name;
|
||||
|
||||
/// For MODIFY TTL
|
||||
ASTPtr ttl;
|
||||
|
||||
@ -84,7 +93,8 @@ struct AlterCommand
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
|
||||
/// Checks that not only metadata touched by that command
|
||||
bool isMutable() const;
|
||||
@ -95,8 +105,9 @@ class Context;
|
||||
class AlterCommands : public std::vector<AlterCommand>
|
||||
{
|
||||
public:
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast,
|
||||
ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
|
||||
/// For storages that don't support MODIFY_ORDER_BY.
|
||||
void apply(ColumnsDescription & columns_description) const;
|
||||
|
54
dbms/src/Storages/ConstraintsDescription.cpp
Normal file
54
dbms/src/Storages/ConstraintsDescription.cpp
Normal file
@ -0,0 +1,54 @@
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
String ConstraintsDescription::toString() const
|
||||
{
|
||||
if (constraints.empty())
|
||||
return {};
|
||||
|
||||
ASTExpressionList list;
|
||||
for (const auto & constraint : constraints)
|
||||
list.children.push_back(constraint);
|
||||
|
||||
return serializeAST(list, true);
|
||||
}
|
||||
|
||||
ConstraintsDescription ConstraintsDescription::parse(const String & str)
|
||||
{
|
||||
if (str.empty())
|
||||
return {};
|
||||
|
||||
ConstraintsDescription res;
|
||||
ParserConstraintDeclarationList parser;
|
||||
ASTPtr list = parseQuery(parser, str, 0);
|
||||
|
||||
for (const auto & constraint : list->children)
|
||||
res.constraints.push_back(std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::Context & context,
|
||||
const DB::NamesAndTypesList & source_columns_) const
|
||||
{
|
||||
ConstraintsExpressions res;
|
||||
res.reserve(constraints.size());
|
||||
for (const auto & constraint : constraints)
|
||||
{
|
||||
// SyntaxAnalyzer::analyze has query as non-const argument so to avoid accidental query changes we clone it
|
||||
ASTPtr expr = constraint->expr->clone();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(expr, source_columns_);
|
||||
res.push_back(ExpressionAnalyzer(constraint->expr->clone(), syntax_result, context).getActions(false));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
26
dbms/src/Storages/ConstraintsDescription.h
Normal file
26
dbms/src/Storages/ConstraintsDescription.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ConstraintsASTs = std::vector<std::shared_ptr<ASTConstraintDeclaration>>;
|
||||
using ConstraintsExpressions = std::vector<ExpressionActionsPtr>;
|
||||
|
||||
struct ConstraintsDescription
|
||||
{
|
||||
ConstraintsASTs constraints;
|
||||
|
||||
ConstraintsDescription() = default;
|
||||
|
||||
bool empty() const { return constraints.empty(); }
|
||||
String toString() const;
|
||||
|
||||
static ConstraintsDescription parse(const String & str);
|
||||
|
||||
ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const;
|
||||
};
|
||||
|
||||
}
|
@ -51,6 +51,11 @@ const IndicesDescription & IStorage::getIndices() const
|
||||
return indices;
|
||||
}
|
||||
|
||||
const ConstraintsDescription & IStorage::getConstraints() const
|
||||
{
|
||||
return constraints;
|
||||
}
|
||||
|
||||
NameAndTypePair IStorage::getColumn(const String & column_name) const
|
||||
{
|
||||
/// By default, we assume that there are no virtual columns in the storage.
|
||||
@ -301,6 +306,11 @@ void IStorage::setIndices(IndicesDescription indices_)
|
||||
indices = std::move(indices_);
|
||||
}
|
||||
|
||||
void IStorage::setConstraints(ConstraintsDescription constraints_)
|
||||
{
|
||||
constraints = std::move(constraints_);
|
||||
}
|
||||
|
||||
bool IStorage::isVirtualColumn(const String & column_name) const
|
||||
{
|
||||
return getColumns().get(column_name).is_virtual;
|
||||
@ -376,8 +386,9 @@ void IStorage::alter(
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Common/ActionLock.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/RWLock.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
|
||||
#include <optional>
|
||||
#include <shared_mutex>
|
||||
@ -107,6 +108,9 @@ public: /// thread-unsafe part. lockStructure must be acquired
|
||||
const ColumnsDescription & getVirtuals() const;
|
||||
const IndicesDescription & getIndices() const;
|
||||
|
||||
const ConstraintsDescription & getConstraints() const;
|
||||
void setConstraints(ConstraintsDescription constraints_);
|
||||
|
||||
/// NOTE: these methods should include virtual columns,
|
||||
/// but should NOT include ALIAS columns (they are treated separately).
|
||||
virtual NameAndTypePair getColumn(const String & column_name) const;
|
||||
@ -143,6 +147,7 @@ private:
|
||||
ColumnsDescription columns; /// combined real and virtual columns
|
||||
const ColumnsDescription virtuals = {};
|
||||
IndicesDescription indices;
|
||||
ConstraintsDescription constraints;
|
||||
|
||||
public:
|
||||
/// Acquire this lock if you need the table structure to remain constant during the execution of
|
||||
|
@ -1,23 +1,6 @@
|
||||
/* Copyright (c) 2018 BlackBerry Limited
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
#pragma once
|
||||
|
||||
#include <limits>
|
||||
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Poco/Condition.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -61,8 +44,8 @@ public:
|
||||
if (isCancelled() || storage->is_dropped)
|
||||
return;
|
||||
IBlockInputStream::cancel(kill);
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
storage->condition.broadcast();
|
||||
std::lock_guard lock(storage->mutex);
|
||||
storage->condition.notify_all();
|
||||
}
|
||||
|
||||
Block getHeader() const override { return storage->getHeader(); }
|
||||
@ -92,14 +75,14 @@ public:
|
||||
|
||||
NonBlockingResult tryRead()
|
||||
{
|
||||
return tryRead_(false);
|
||||
return tryReadImpl(false);
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
/// try reading
|
||||
return tryRead_(true).first;
|
||||
return tryReadImpl(true).first;
|
||||
}
|
||||
|
||||
/** tryRead method attempts to read a block in either blocking
|
||||
@ -107,7 +90,7 @@ protected:
|
||||
* then method return empty block with flag set to false
|
||||
* to indicate that method would block to get the next block.
|
||||
*/
|
||||
NonBlockingResult tryRead_(bool blocking)
|
||||
NonBlockingResult tryReadImpl(bool blocking)
|
||||
{
|
||||
Block res;
|
||||
|
||||
@ -118,7 +101,7 @@ protected:
|
||||
/// If blocks were never assigned get blocks
|
||||
if (!blocks)
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
std::lock_guard lock(storage->mutex);
|
||||
if (!active)
|
||||
return { Block(), false };
|
||||
blocks = (*blocks_ptr);
|
||||
@ -135,7 +118,7 @@ protected:
|
||||
if (it == end)
|
||||
{
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
std::unique_lock lock(storage->mutex);
|
||||
if (!active)
|
||||
return { Block(), false };
|
||||
/// If we are done iterating over our blocks
|
||||
@ -162,7 +145,10 @@ protected:
|
||||
while (true)
|
||||
{
|
||||
UInt64 timestamp_usec = static_cast<UInt64>(timestamp.epochMicroseconds());
|
||||
bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast<UInt64>(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000);
|
||||
|
||||
/// Or spurious wakeup.
|
||||
bool signaled = std::cv_status::no_timeout == storage->condition.wait_for(lock,
|
||||
std::chrono::microseconds(std::max(UInt64(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec))));
|
||||
|
||||
if (isCancelled() || storage->is_dropped)
|
||||
{
|
||||
@ -181,7 +167,7 @@ protected:
|
||||
}
|
||||
}
|
||||
}
|
||||
return tryRead_(blocking);
|
||||
return tryReadImpl(blocking);
|
||||
}
|
||||
|
||||
res = *it;
|
74
dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h
Normal file
74
dbms/src/Storages/LiveView/LiveViewBlockOutputStream.h
Normal file
@ -0,0 +1,74 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class LiveViewBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
new_blocks = std::make_shared<Blocks>();
|
||||
new_blocks_metadata = std::make_shared<BlocksMetadata>();
|
||||
new_hash = std::make_shared<SipHash>();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
UInt128 key;
|
||||
String key_str;
|
||||
|
||||
new_hash->get128(key.low, key.high);
|
||||
key_str = key.toHexString();
|
||||
|
||||
std::lock_guard lock(storage.mutex);
|
||||
|
||||
if (storage.getBlocksHashKey() != key_str)
|
||||
{
|
||||
new_blocks_metadata->hash = key_str;
|
||||
new_blocks_metadata->version = storage.getBlocksVersion() + 1;
|
||||
|
||||
for (auto & block : *new_blocks)
|
||||
{
|
||||
block.insert({DataTypeUInt64().createColumnConst(
|
||||
block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(),
|
||||
std::make_shared<DataTypeUInt64>(),
|
||||
"_version"});
|
||||
}
|
||||
|
||||
(*storage.blocks_ptr) = new_blocks;
|
||||
(*storage.blocks_metadata_ptr) = new_blocks_metadata;
|
||||
|
||||
storage.condition.notify_all();
|
||||
}
|
||||
|
||||
new_blocks.reset();
|
||||
new_blocks_metadata.reset();
|
||||
new_hash.reset();
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
new_blocks->push_back(block);
|
||||
block.updateHash(*new_hash);
|
||||
}
|
||||
|
||||
Block getHeader() const override { return storage.getHeader(); }
|
||||
|
||||
private:
|
||||
using SipHashPtr = std::shared_ptr<SipHash>;
|
||||
|
||||
BlocksPtr new_blocks;
|
||||
BlocksMetadataPtr new_blocks_metadata;
|
||||
SipHashPtr new_hash;
|
||||
StorageLiveView & storage;
|
||||
};
|
||||
|
||||
}
|
@ -12,9 +12,9 @@ limitations under the License. */
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <optional>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -9,11 +9,9 @@ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <limits>
|
||||
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Poco/Condition.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -21,7 +19,7 @@ limitations under the License. */
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Storages/StorageLiveView.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -66,8 +64,8 @@ public:
|
||||
if (isCancelled() || storage->is_dropped)
|
||||
return;
|
||||
IBlockInputStream::cancel(kill);
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
storage->condition.broadcast();
|
||||
std::lock_guard lock(storage->mutex);
|
||||
storage->condition.notify_all();
|
||||
}
|
||||
|
||||
Block getHeader() const override
|
||||
@ -103,7 +101,7 @@ public:
|
||||
|
||||
NonBlockingResult tryRead()
|
||||
{
|
||||
return tryRead_(false);
|
||||
return tryReadImpl(false);
|
||||
}
|
||||
|
||||
Block getEventBlock()
|
||||
@ -120,7 +118,7 @@ protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
/// try reading
|
||||
return tryRead_(true).first;
|
||||
return tryReadImpl(true).first;
|
||||
}
|
||||
|
||||
/** tryRead method attempts to read a block in either blocking
|
||||
@ -128,7 +126,7 @@ protected:
|
||||
* then method return empty block with flag set to false
|
||||
* to indicate that method would block to get the next block.
|
||||
*/
|
||||
NonBlockingResult tryRead_(bool blocking)
|
||||
NonBlockingResult tryReadImpl(bool blocking)
|
||||
{
|
||||
if (has_limit && num_updates == static_cast<Int64>(limit))
|
||||
{
|
||||
@ -137,7 +135,7 @@ protected:
|
||||
/// If blocks were never assigned get blocks
|
||||
if (!blocks)
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
std::lock_guard lock(storage->mutex);
|
||||
if (!active)
|
||||
return { Block(), false };
|
||||
blocks = (*blocks_ptr);
|
||||
@ -155,7 +153,7 @@ protected:
|
||||
if (it == end)
|
||||
{
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(storage->mutex);
|
||||
std::unique_lock lock(storage->mutex);
|
||||
if (!active)
|
||||
return { Block(), false };
|
||||
/// If we are done iterating over our blocks
|
||||
@ -183,7 +181,10 @@ protected:
|
||||
while (true)
|
||||
{
|
||||
UInt64 timestamp_usec = static_cast<UInt64>(timestamp.epochMicroseconds());
|
||||
bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast<UInt64>(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000);
|
||||
|
||||
/// Or spurious wakeup.
|
||||
bool signaled = std::cv_status::no_timeout == storage->condition.wait_for(lock,
|
||||
std::chrono::microseconds(std::max(UInt64(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec))));
|
||||
|
||||
if (isCancelled() || storage->is_dropped)
|
||||
{
|
||||
@ -202,7 +203,7 @@ protected:
|
||||
}
|
||||
}
|
||||
}
|
||||
return tryRead_(blocking);
|
||||
return tryReadImpl(blocking);
|
||||
}
|
||||
|
||||
// move right to the end
|
@ -9,6 +9,7 @@ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTWatchQuery.h>
|
||||
@ -17,15 +18,23 @@ limitations under the License. */
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <DataStreams/NullBlockInputStream.h>
|
||||
#include <DataStreams/LiveViewBlockInputStream.h>
|
||||
#include <DataStreams/LiveViewEventsBlockInputStream.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/BlocksBlockInputStream.h>
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
#include <Storages/LiveView/LiveViewBlockInputStream.h>
|
||||
#include <Storages/LiveView/LiveViewBlockOutputStream.h>
|
||||
#include <Storages/LiveView/LiveViewEventsBlockInputStream.h>
|
||||
#include <Storages/LiveView/ProxyStorage.h>
|
||||
|
||||
#include <Storages/StorageLiveView.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/ProxyStorage.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
@ -40,6 +49,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int TABLE_WAS_NOT_DROPPED;
|
||||
extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name)
|
||||
@ -98,6 +108,110 @@ static void checkAllowedQueries(const ASTSelectQuery & query)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void StorageLiveView::writeIntoLiveView(
|
||||
StorageLiveView & live_view,
|
||||
const Block & block,
|
||||
const Context & context)
|
||||
{
|
||||
BlockOutputStreamPtr output = std::make_shared<LiveViewBlockOutputStream>(live_view);
|
||||
|
||||
/// Check if live view has any readers if not
|
||||
/// just reset blocks to empty and do nothing else
|
||||
/// When first reader comes the blocks will be read.
|
||||
{
|
||||
std::lock_guard lock(live_view.mutex);
|
||||
if (!live_view.hasActiveUsers())
|
||||
{
|
||||
live_view.reset();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
bool is_block_processed = false;
|
||||
BlockInputStreams from;
|
||||
BlocksPtrs mergeable_blocks;
|
||||
BlocksPtr new_mergeable_blocks = std::make_shared<Blocks>();
|
||||
|
||||
{
|
||||
std::lock_guard lock(live_view.mutex);
|
||||
|
||||
mergeable_blocks = live_view.getMergeableBlocks();
|
||||
if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh)
|
||||
{
|
||||
mergeable_blocks = std::make_shared<std::vector<BlocksPtr>>();
|
||||
BlocksPtr base_mergeable_blocks = std::make_shared<Blocks>();
|
||||
InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names());
|
||||
auto view_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
interpreter.execute().in);
|
||||
while (Block this_block = view_mergeable_stream->read())
|
||||
base_mergeable_blocks->push_back(this_block);
|
||||
mergeable_blocks->push_back(base_mergeable_blocks);
|
||||
live_view.setMergeableBlocks(mergeable_blocks);
|
||||
|
||||
/// Create from streams
|
||||
for (auto & blocks_ : *mergeable_blocks)
|
||||
{
|
||||
if (blocks_->empty())
|
||||
continue;
|
||||
auto sample_block = blocks_->front().cloneEmpty();
|
||||
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
|
||||
from.push_back(std::move(stream));
|
||||
}
|
||||
|
||||
is_block_processed = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!is_block_processed)
|
||||
{
|
||||
auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName());
|
||||
BlockInputStreams streams = {std::make_shared<OneBlockInputStream>(block)};
|
||||
auto proxy_storage = std::make_shared<ProxyStorage>(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns);
|
||||
InterpreterSelectQuery select_block(live_view.getInnerQuery(),
|
||||
context, proxy_storage,
|
||||
QueryProcessingStage::WithMergeableState);
|
||||
auto data_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
select_block.execute().in);
|
||||
while (Block this_block = data_mergeable_stream->read())
|
||||
new_mergeable_blocks->push_back(this_block);
|
||||
|
||||
if (new_mergeable_blocks->empty())
|
||||
return;
|
||||
|
||||
{
|
||||
std::lock_guard lock(live_view.mutex);
|
||||
|
||||
mergeable_blocks = live_view.getMergeableBlocks();
|
||||
mergeable_blocks->push_back(new_mergeable_blocks);
|
||||
|
||||
/// Create from streams
|
||||
for (auto & blocks_ : *mergeable_blocks)
|
||||
{
|
||||
if (blocks_->empty())
|
||||
continue;
|
||||
auto sample_block = blocks_->front().cloneEmpty();
|
||||
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
|
||||
from.push_back(std::move(stream));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName());
|
||||
auto proxy_storage = std::make_shared<ProxyStorage>(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState);
|
||||
InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete);
|
||||
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
||||
|
||||
/// Squashing is needed here because the 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).
|
||||
data = std::make_shared<SquashingBlockInputStream>(
|
||||
data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes);
|
||||
|
||||
copyData(*data, *output);
|
||||
}
|
||||
|
||||
|
||||
StorageLiveView::StorageLiveView(
|
||||
const String & table_name_,
|
||||
const String & database_name_,
|
||||
@ -259,11 +373,10 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout)
|
||||
{
|
||||
while (1)
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex,
|
||||
timeout * 1000))
|
||||
std::unique_lock lock(no_users_thread_mutex);
|
||||
if (!no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return no_users_thread_wakeup; }))
|
||||
{
|
||||
noUsersThreadWakeUp = false;
|
||||
no_users_thread_wakeup = false;
|
||||
if (shutdown_called)
|
||||
return;
|
||||
if (hasUsers())
|
||||
@ -301,7 +414,7 @@ void StorageLiveView::noUsersThread(const UInt64 & timeout)
|
||||
void StorageLiveView::startNoUsersThread(const UInt64 & timeout)
|
||||
{
|
||||
bool expected = false;
|
||||
if (!startnousersthread_called.compare_exchange_strong(expected, true))
|
||||
if (!start_no_users_thread_called.compare_exchange_strong(expected, true))
|
||||
return;
|
||||
|
||||
if (is_dropped)
|
||||
@ -312,20 +425,20 @@ void StorageLiveView::startNoUsersThread(const UInt64 & timeout)
|
||||
if (no_users_thread.joinable())
|
||||
{
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
noUsersThreadWakeUp = true;
|
||||
noUsersThreadCondition.signal();
|
||||
std::lock_guard lock(no_users_thread_mutex);
|
||||
no_users_thread_wakeup = true;
|
||||
no_users_thread_condition.notify_one();
|
||||
}
|
||||
no_users_thread.join();
|
||||
}
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
noUsersThreadWakeUp = false;
|
||||
std::lock_guard lock(no_users_thread_mutex);
|
||||
no_users_thread_wakeup = false;
|
||||
}
|
||||
if (!is_dropped)
|
||||
no_users_thread = std::thread(&StorageLiveView::noUsersThread, this, timeout);
|
||||
}
|
||||
startnousersthread_called = false;
|
||||
start_no_users_thread_called = false;
|
||||
}
|
||||
|
||||
void StorageLiveView::startup()
|
||||
@ -341,13 +454,13 @@ void StorageLiveView::shutdown()
|
||||
|
||||
if (no_users_thread.joinable())
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
noUsersThreadWakeUp = true;
|
||||
noUsersThreadCondition.signal();
|
||||
std::lock_guard lock(no_users_thread_mutex);
|
||||
no_users_thread_wakeup = true;
|
||||
no_users_thread_condition.notify_one();
|
||||
/// Must detach the no users thread
|
||||
/// as we can't join it as it will result
|
||||
/// in a deadlock
|
||||
no_users_thread.detach();
|
||||
no_users_thread.detach(); /// TODO Not viable at all.
|
||||
}
|
||||
}
|
||||
|
||||
@ -361,18 +474,19 @@ void StorageLiveView::drop()
|
||||
global_context.removeDependency(
|
||||
DatabaseAndTableName(select_database_name, select_table_name),
|
||||
DatabaseAndTableName(database_name, table_name));
|
||||
Poco::FastMutex::ScopedLock lock(mutex);
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
is_dropped = true;
|
||||
condition.broadcast();
|
||||
condition.notify_all();
|
||||
}
|
||||
|
||||
void StorageLiveView::refresh(const Context & context)
|
||||
{
|
||||
auto alter_lock = lockAlterIntention(context.getCurrentQueryId());
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (getNewBlocks())
|
||||
condition.broadcast();
|
||||
condition.notify_all();
|
||||
}
|
||||
}
|
||||
|
||||
@ -387,11 +501,11 @@ BlockInputStreams StorageLiveView::read(
|
||||
/// add user to the blocks_ptr
|
||||
std::shared_ptr<BlocksPtr> stream_blocks_ptr = blocks_ptr;
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!(*blocks_ptr))
|
||||
{
|
||||
if (getNewBlocks())
|
||||
condition.broadcast();
|
||||
condition.notify_all();
|
||||
}
|
||||
}
|
||||
return { std::make_shared<BlocksBlockInputStream>(stream_blocks_ptr, getHeader()) };
|
||||
@ -423,17 +537,17 @@ BlockInputStreams StorageLiveView::watch(
|
||||
|
||||
if (no_users_thread.joinable())
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
noUsersThreadWakeUp = true;
|
||||
noUsersThreadCondition.signal();
|
||||
std::lock_guard lock(no_users_thread_mutex);
|
||||
no_users_thread_wakeup = true;
|
||||
no_users_thread_condition.notify_one();
|
||||
}
|
||||
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!(*blocks_ptr))
|
||||
{
|
||||
if (getNewBlocks())
|
||||
condition.broadcast();
|
||||
condition.notify_all();
|
||||
}
|
||||
}
|
||||
|
||||
@ -448,17 +562,17 @@ BlockInputStreams StorageLiveView::watch(
|
||||
|
||||
if (no_users_thread.joinable())
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
|
||||
noUsersThreadWakeUp = true;
|
||||
noUsersThreadCondition.signal();
|
||||
std::lock_guard lock(no_users_thread_mutex);
|
||||
no_users_thread_wakeup = true;
|
||||
no_users_thread_condition.notify_one();
|
||||
}
|
||||
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!(*blocks_ptr))
|
||||
{
|
||||
if (getNewBlocks())
|
||||
condition.broadcast();
|
||||
condition.notify_all();
|
||||
}
|
||||
}
|
||||
|
||||
@ -472,6 +586,9 @@ void registerStorageLiveView(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage("LiveView", [](const StorageFactory::Arguments & args)
|
||||
{
|
||||
if (!args.local_context.getSettingsRef().allow_experimental_live_view)
|
||||
throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
|
||||
return StorageLiveView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns);
|
||||
});
|
||||
}
|
184
dbms/src/Storages/LiveView/StorageLiveView.h
Normal file
184
dbms/src/Storages/LiveView/StorageLiveView.h
Normal file
@ -0,0 +1,184 @@
|
||||
/* Copyright (c) 2018 BlackBerry Limited
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct BlocksMetadata
|
||||
{
|
||||
String hash;
|
||||
UInt64 version;
|
||||
};
|
||||
|
||||
class IAST;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using BlocksMetadataPtr = std::shared_ptr<BlocksMetadata>;
|
||||
|
||||
class StorageLiveView : public ext::shared_ptr_helper<StorageLiveView>, public IStorage
|
||||
{
|
||||
friend struct ext::shared_ptr_helper<StorageLiveView>;
|
||||
friend class LiveViewBlockInputStream;
|
||||
friend class LiveViewEventsBlockInputStream;
|
||||
friend class LiveViewBlockOutputStream;
|
||||
|
||||
public:
|
||||
~StorageLiveView() override;
|
||||
String getName() const override { return "LiveView"; }
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
String getSelectDatabaseName() const { return select_database_name; }
|
||||
String getSelectTableName() const { return select_table_name; }
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
bool hasColumn(const String & column_name) const override;
|
||||
|
||||
// const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||
ASTPtr getInnerQuery() const { return inner_query->clone(); }
|
||||
|
||||
/// It is passed inside the query and solved at its level.
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
|
||||
bool isTemporary() { return is_temporary; }
|
||||
|
||||
/// Check if we have any readers
|
||||
/// must be called with mutex locked
|
||||
bool hasUsers()
|
||||
{
|
||||
return blocks_ptr.use_count() > 1;
|
||||
}
|
||||
|
||||
/// Check we have any active readers
|
||||
/// must be called with mutex locked
|
||||
bool hasActiveUsers()
|
||||
{
|
||||
return active_ptr.use_count() > 1;
|
||||
}
|
||||
/// Background thread for temporary tables
|
||||
/// which drops this table if there are no users
|
||||
void startNoUsersThread(const UInt64 & timeout);
|
||||
std::mutex no_users_thread_mutex;
|
||||
bool no_users_thread_wakeup{false};
|
||||
std::condition_variable no_users_thread_condition;
|
||||
/// Get blocks hash
|
||||
/// must be called with mutex locked
|
||||
String getBlocksHashKey()
|
||||
{
|
||||
if (*blocks_metadata_ptr)
|
||||
return (*blocks_metadata_ptr)->hash;
|
||||
return {};
|
||||
}
|
||||
/// Get blocks version
|
||||
/// must be called with mutex locked
|
||||
UInt64 getBlocksVersion()
|
||||
{
|
||||
if (*blocks_metadata_ptr)
|
||||
return (*blocks_metadata_ptr)->version;
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Reset blocks
|
||||
/// must be called with mutex locked
|
||||
void reset()
|
||||
{
|
||||
(*blocks_ptr).reset();
|
||||
if (*blocks_metadata_ptr)
|
||||
(*blocks_metadata_ptr)->hash.clear();
|
||||
mergeable_blocks.reset();
|
||||
}
|
||||
|
||||
void checkTableCanBeDropped() const override;
|
||||
void drop() override;
|
||||
void startup() override;
|
||||
void shutdown() override;
|
||||
|
||||
void refresh(const Context & context);
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
BlockInputStreams watch(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
std::shared_ptr<BlocksPtr> getBlocksPtr() { return blocks_ptr; }
|
||||
BlocksPtrs getMergeableBlocks() { return mergeable_blocks; }
|
||||
void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; }
|
||||
std::shared_ptr<bool> getActivePtr() { return active_ptr; }
|
||||
|
||||
/// Read new data blocks that store query result
|
||||
bool getNewBlocks();
|
||||
|
||||
Block getHeader() const;
|
||||
|
||||
static void writeIntoLiveView(
|
||||
StorageLiveView & live_view,
|
||||
const Block & block,
|
||||
const Context & context);
|
||||
|
||||
private:
|
||||
String select_database_name;
|
||||
String select_table_name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
ASTPtr inner_query;
|
||||
Context & global_context;
|
||||
bool is_temporary {false};
|
||||
mutable Block sample_block;
|
||||
|
||||
/// Mutex for the blocks and ready condition
|
||||
std::mutex mutex;
|
||||
/// New blocks ready condition to broadcast to readers
|
||||
/// that new blocks are available
|
||||
std::condition_variable condition;
|
||||
|
||||
/// Active users
|
||||
std::shared_ptr<bool> active_ptr;
|
||||
/// Current data blocks that store query result
|
||||
std::shared_ptr<BlocksPtr> blocks_ptr;
|
||||
/// Current data blocks metadata
|
||||
std::shared_ptr<BlocksMetadataPtr> blocks_metadata_ptr;
|
||||
BlocksPtrs mergeable_blocks;
|
||||
|
||||
void noUsersThread(const UInt64 & timeout);
|
||||
std::thread no_users_thread;
|
||||
std::atomic<bool> shutdown_called{false};
|
||||
std::atomic<bool> start_no_users_thread_called{false};
|
||||
UInt64 temporary_live_view_timeout;
|
||||
|
||||
StorageLiveView(
|
||||
const String & table_name_,
|
||||
const String & database_name_,
|
||||
Context & local_context,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns
|
||||
);
|
||||
};
|
||||
|
||||
}
|
@ -87,6 +87,8 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_MUNMAP;
|
||||
extern const int CANNOT_MREMAP;
|
||||
extern const int BAD_TTL_EXPRESSION;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int BAD_DATA_PART_NAME;
|
||||
}
|
||||
|
||||
|
||||
@ -94,6 +96,7 @@ MergeTreeData::MergeTreeData(
|
||||
const String & database_, const String & table_,
|
||||
const String & full_path_, const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
@ -120,7 +123,8 @@ MergeTreeData::MergeTreeData(
|
||||
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
|
||||
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
|
||||
{
|
||||
setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_);
|
||||
setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_);
|
||||
setConstraints(constraints_);
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
merging_params.check(getColumns().getAllPhysical());
|
||||
@ -229,9 +233,10 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::setPrimaryKeyIndicesAndColumns(
|
||||
void MergeTreeData::setProperties(
|
||||
const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
|
||||
const ColumnsDescription & new_columns, const IndicesDescription & indices_description, bool only_check)
|
||||
const ColumnsDescription & new_columns, const IndicesDescription & indices_description,
|
||||
const ConstraintsDescription & constraints_description, bool only_check)
|
||||
{
|
||||
if (!new_order_by_ast)
|
||||
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -400,6 +405,8 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns(
|
||||
setIndices(indices_description);
|
||||
skip_indices = std::move(new_indices);
|
||||
|
||||
setConstraints(constraints_description);
|
||||
|
||||
primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr;
|
||||
sorting_key_and_skip_indices_expr = new_indices_with_sorting_key_expr;
|
||||
}
|
||||
@ -1219,11 +1226,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
commands.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
if (getIndices().empty() && !new_indices.empty() &&
|
||||
!context.getSettingsRef().allow_experimental_data_skipping_indices)
|
||||
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
||||
@ -1305,8 +1312,8 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
|
||||
}
|
||||
}
|
||||
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast,
|
||||
new_columns, new_indices, /* only_check = */ true);
|
||||
setProperties(new_order_by_ast, new_primary_key_ast,
|
||||
new_columns, new_indices, new_constraints, /* only_check = */ true);
|
||||
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true);
|
||||
|
||||
@ -1766,6 +1773,52 @@ MergeTreeData::AlterDataPartTransaction::~AlterDataPartTransaction()
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name)
|
||||
{
|
||||
old_and_new_names.push_back({old_name, new_name});
|
||||
}
|
||||
|
||||
void MergeTreeData::PartsTemporaryRename::tryRenameAll()
|
||||
{
|
||||
renamed = true;
|
||||
for (size_t i = 0; i < old_and_new_names.size(); ++i)
|
||||
{
|
||||
try
|
||||
{
|
||||
const auto & names = old_and_new_names[i];
|
||||
if (names.first.empty() || names.second.empty())
|
||||
throw DB::Exception("Empty part name. Most likely it's a bug.", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
Poco::File(base_dir + names.first).renameTo(base_dir + names.second);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
old_and_new_names.resize(i);
|
||||
LOG_WARNING(storage.log, "Cannot rename parts to perform operation on them: " << getCurrentExceptionMessage(false));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename()
|
||||
{
|
||||
// TODO what if server had crashed before this destructor was called?
|
||||
if (!renamed)
|
||||
return;
|
||||
for (const auto & names : old_and_new_names)
|
||||
{
|
||||
if (names.first.empty())
|
||||
continue;
|
||||
try
|
||||
{
|
||||
Poco::File(base_dir + names.second).renameTo(base_dir + names.first);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace(
|
||||
const MergeTreePartInfo & new_part_info,
|
||||
@ -2386,6 +2439,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St
|
||||
{
|
||||
MutableDataPartPtr part = std::make_shared<DataPart>(*this, Poco::Path(relative_path).getFileName());
|
||||
part->relative_path = relative_path;
|
||||
loadPartAndFixMetadata(part);
|
||||
return part;
|
||||
}
|
||||
|
||||
void MergeTreeData::loadPartAndFixMetadata(MutableDataPartPtr part)
|
||||
{
|
||||
String full_part_path = part->getFullPath();
|
||||
|
||||
/// Earlier the list of columns was written incorrectly. Delete it and re-create.
|
||||
@ -2407,8 +2466,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St
|
||||
|
||||
Poco::File(full_part_path + "checksums.txt.tmp").renameTo(full_part_path + "checksums.txt");
|
||||
}
|
||||
|
||||
return part;
|
||||
}
|
||||
|
||||
|
||||
@ -2628,26 +2685,124 @@ MergeTreeData::getDetachedParts() const
|
||||
res.emplace_back();
|
||||
auto & part = res.back();
|
||||
|
||||
/// First, try to parse as <part_name>.
|
||||
if (MergeTreePartInfo::tryParsePartName(dir_name, &part, format_version))
|
||||
continue;
|
||||
|
||||
/// Next, as <prefix>_<partname>. Use entire name as prefix if it fails.
|
||||
part.prefix = dir_name;
|
||||
const auto first_separator = dir_name.find_first_of('_');
|
||||
if (first_separator == String::npos)
|
||||
continue;
|
||||
|
||||
const auto part_name = dir_name.substr(first_separator + 1,
|
||||
dir_name.size() - first_separator - 1);
|
||||
if (!MergeTreePartInfo::tryParsePartName(part_name, &part, format_version))
|
||||
continue;
|
||||
|
||||
part.prefix = dir_name.substr(0, first_separator);
|
||||
DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeData::validateDetachedPartName(const String & name) const
|
||||
{
|
||||
if (name.find('/') != std::string::npos || name == "." || name == "..")
|
||||
throw DB::Exception("Invalid part name", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
Poco::File detached_part_dir(full_path + "detached/" + name);
|
||||
if (!detached_part_dir.exists())
|
||||
throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME);
|
||||
|
||||
if (startsWith(name, "attaching_") || startsWith(name, "deleting_"))
|
||||
throw DB::Exception("Cannot drop part " + name + ": "
|
||||
"most likely it is used by another DROP or ATTACH query.",
|
||||
ErrorCodes::BAD_DATA_PART_NAME);
|
||||
}
|
||||
|
||||
void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Context & context)
|
||||
{
|
||||
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
|
||||
|
||||
if (part)
|
||||
{
|
||||
String part_name = partition->as<ASTLiteral &>().value.safeGet<String>();
|
||||
validateDetachedPartName(part_name);
|
||||
renamed_parts.addPart(part_name, "deleting_" + part_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
String partition_id = getPartitionIDFromQuery(partition, context);
|
||||
DetachedPartsInfo detached_parts = getDetachedParts();
|
||||
for (const auto & part_info : detached_parts)
|
||||
if (part_info.valid_name && part_info.partition_id == partition_id
|
||||
&& part_info.prefix != "attaching" && part_info.prefix != "deleting")
|
||||
renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Will drop " << renamed_parts.old_and_new_names.size() << " detached parts.");
|
||||
|
||||
renamed_parts.tryRenameAll();
|
||||
|
||||
for (auto & names : renamed_parts.old_and_new_names)
|
||||
{
|
||||
Poco::File(renamed_parts.base_dir + names.second).remove(true);
|
||||
LOG_DEBUG(log, "Dropped detached part " << names.first);
|
||||
names.first.clear();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part,
|
||||
const Context & context, PartsTemporaryRename & renamed_parts)
|
||||
{
|
||||
String source_dir = "detached/";
|
||||
|
||||
/// Let's compose a list of parts that should be added.
|
||||
if (attach_part)
|
||||
{
|
||||
String part_id = partition->as<ASTLiteral &>().value.safeGet<String>();
|
||||
validateDetachedPartName(part_id);
|
||||
renamed_parts.addPart(part_id, "attaching_" + part_id);
|
||||
}
|
||||
else
|
||||
{
|
||||
String partition_id = getPartitionIDFromQuery(partition, context);
|
||||
LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
|
||||
ActiveDataPartSet active_parts(format_version);
|
||||
|
||||
std::set<String> part_names;
|
||||
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
|
||||
{
|
||||
String name = it.name();
|
||||
MergeTreePartInfo part_info;
|
||||
// TODO what if name contains "_tryN" suffix?
|
||||
/// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored
|
||||
if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version))
|
||||
continue;
|
||||
if (part_info.partition_id != partition_id)
|
||||
continue;
|
||||
LOG_DEBUG(log, "Found part " << name);
|
||||
active_parts.add(name);
|
||||
part_names.insert(name);
|
||||
}
|
||||
LOG_DEBUG(log, active_parts.size() << " of them are active");
|
||||
|
||||
/// Inactive parts rename so they can not be attached in case of repeated ATTACH.
|
||||
for (const auto & name : part_names)
|
||||
{
|
||||
String containing_part = active_parts.getContainingPart(name);
|
||||
if (!containing_part.empty() && containing_part != name)
|
||||
// TODO maybe use PartsTemporaryRename here?
|
||||
Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name);
|
||||
else
|
||||
renamed_parts.addPart(name, "attaching_" + name);
|
||||
}
|
||||
}
|
||||
|
||||
/// Try to rename all parts before attaching to prevent race with DROP DETACHED and another ATTACH.
|
||||
renamed_parts.tryRenameAll();
|
||||
|
||||
/// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
|
||||
LOG_DEBUG(log, "Checking parts");
|
||||
MutableDataPartsVector loaded_parts;
|
||||
loaded_parts.reserve(renamed_parts.old_and_new_names.size());
|
||||
for (const auto & part_names : renamed_parts.old_and_new_names)
|
||||
{
|
||||
LOG_DEBUG(log, "Checking part " << part_names.second);
|
||||
MutableDataPartPtr part = std::make_shared<DataPart>(*this, part_names.first);
|
||||
part->relative_path = source_dir + part_names.second;
|
||||
loadPartAndFixMetadata(part);
|
||||
loaded_parts.push_back(part);
|
||||
}
|
||||
|
||||
return loaded_parts;
|
||||
}
|
||||
|
||||
MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const
|
||||
{
|
||||
DataParts res;
|
||||
|
@ -249,6 +249,23 @@ public:
|
||||
|
||||
using AlterDataPartTransactionPtr = std::unique_ptr<AlterDataPartTransaction>;
|
||||
|
||||
struct PartsTemporaryRename : private boost::noncopyable
|
||||
{
|
||||
PartsTemporaryRename(const MergeTreeData & storage_, const String & base_dir_) : storage(storage_), base_dir(base_dir_) {}
|
||||
|
||||
void addPart(const String & old_name, const String & new_name);
|
||||
|
||||
/// Renames part from old_name to new_name
|
||||
void tryRenameAll();
|
||||
|
||||
/// Renames all added parts from new_name to old_name if old name is not empty
|
||||
~PartsTemporaryRename();
|
||||
|
||||
const MergeTreeData & storage;
|
||||
String base_dir;
|
||||
std::vector<std::pair<String, String>> old_and_new_names;
|
||||
bool renamed = false;
|
||||
};
|
||||
|
||||
/// Parameters for various modes.
|
||||
struct MergingParams
|
||||
@ -305,6 +322,7 @@ public:
|
||||
const String & full_path_,
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
@ -388,7 +406,14 @@ public:
|
||||
DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const;
|
||||
|
||||
/// Returns all detached parts
|
||||
std::vector<DetachedPartInfo> getDetachedParts() const;
|
||||
DetachedPartsInfo getDetachedParts() const;
|
||||
|
||||
void validateDetachedPartName(const String & name) const;
|
||||
|
||||
void dropDetached(const ASTPtr & partition, bool part, const Context & context);
|
||||
|
||||
MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part,
|
||||
const Context & context, PartsTemporaryRename & renamed_parts);
|
||||
|
||||
/// Returns Committed parts
|
||||
DataParts getDataParts() const;
|
||||
@ -536,6 +561,7 @@ public:
|
||||
|
||||
/// Check that the part is not broken and calculate the checksums for it if they are not present.
|
||||
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
|
||||
void loadPartAndFixMetadata(MutableDataPartPtr part);
|
||||
|
||||
/** Create local backup (snapshot) for parts with specified prefix.
|
||||
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
|
||||
@ -760,9 +786,10 @@ protected:
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
|
||||
void setPrimaryKeyIndicesAndColumns(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
|
||||
void setProperties(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
|
||||
const ColumnsDescription & new_columns,
|
||||
const IndicesDescription & indices_description, bool only_check = false);
|
||||
const IndicesDescription & indices_description,
|
||||
const ConstraintsDescription & constraints_description, bool only_check = false);
|
||||
|
||||
void initPartitionKey();
|
||||
|
||||
|
@ -157,7 +157,14 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz
|
||||
|
||||
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation()
|
||||
{
|
||||
return static_cast<UInt64>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_RESERVE);
|
||||
size_t total_threads_in_pool = pool.getNumberOfThreads();
|
||||
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
|
||||
|
||||
/// Allow mutations only if there are enough threads, leave free threads for merges else
|
||||
if (total_threads_in_pool - busy_threads_in_pool >= data.settings.number_of_free_entries_in_pool_to_execute_mutation)
|
||||
return static_cast<UInt64>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_RESERVE);
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
|
@ -142,10 +142,7 @@ MergeTreeDataPart::MergeTreeDataPart(MergeTreeData & storage_, const String & na
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeDataPart::MergeTreeDataPart(
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_)
|
||||
MergeTreeDataPart::MergeTreeDataPart(const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_)
|
||||
: storage(storage_)
|
||||
, name(name_)
|
||||
, info(info_)
|
||||
@ -367,6 +364,8 @@ void MergeTreeDataPart::remove() const
|
||||
* And a race condition can happen that will lead to "File not found" error here.
|
||||
*/
|
||||
|
||||
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
|
||||
|
||||
String from = storage.full_path + relative_path;
|
||||
String to = storage.full_path + "delete_tmp_" + name;
|
||||
|
||||
|
@ -188,4 +188,30 @@ String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) con
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
bool DetachedPartInfo::tryParseDetachedPartName(const String & dir_name, DetachedPartInfo & part_info,
|
||||
MergeTreeDataFormatVersion format_version)
|
||||
{
|
||||
part_info.dir_name = dir_name;
|
||||
|
||||
/// First, try to parse as <part_name>.
|
||||
// TODO what if tryParsePartName will parse prefix as partition_id? It can happen if dir_name doesn't contain mutation number at the end
|
||||
if (MergeTreePartInfo::tryParsePartName(dir_name, &part_info, format_version))
|
||||
return part_info.valid_name = true;
|
||||
|
||||
/// Next, as <prefix>_<partname>. Use entire name as prefix if it fails.
|
||||
part_info.prefix = dir_name;
|
||||
const auto first_separator = dir_name.find_first_of('_');
|
||||
if (first_separator == String::npos)
|
||||
return part_info.valid_name = false;
|
||||
|
||||
// TODO what if <prefix> contains '_'?
|
||||
const auto part_name = dir_name.substr(first_separator + 1,
|
||||
dir_name.size() - first_separator - 1);
|
||||
if (!MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version))
|
||||
return part_info.valid_name = false;
|
||||
|
||||
part_info.prefix = dir_name.substr(0, first_separator);
|
||||
return part_info.valid_name = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -92,7 +92,15 @@ struct MergeTreePartInfo
|
||||
/// addition to the above fields.
|
||||
struct DetachedPartInfo : public MergeTreePartInfo
|
||||
{
|
||||
String dir_name;
|
||||
String prefix;
|
||||
|
||||
/// If false, MergeTreePartInfo is in invalid state (directory name was not successfully parsed).
|
||||
bool valid_name;
|
||||
|
||||
static bool tryParseDetachedPartName(const String & dir_name, DetachedPartInfo & part_info, MergeTreeDataFormatVersion format_version);
|
||||
};
|
||||
|
||||
using DetachedPartsInfo = std::vector<DetachedPartInfo>;
|
||||
|
||||
}
|
||||
|
@ -30,8 +30,10 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
|
||||
/** Merge settings. */ \
|
||||
M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).") \
|
||||
M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).") \
|
||||
M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging parts are allowed simultaneously in ReplicatedMergeTree queue.") \
|
||||
M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \
|
||||
M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \
|
||||
M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.") \
|
||||
M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"") \
|
||||
M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.") \
|
||||
M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.") \
|
||||
\
|
||||
|
@ -956,15 +956,19 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
|
||||
return false;
|
||||
}
|
||||
|
||||
/** Execute merge only if there are enough free threads in background pool to do merges of that size.
|
||||
* But if all threads are free (maximal size of merge is allowed) then execute any merge,
|
||||
* (because it may be ordered by OPTIMIZE or early with differrent settings).
|
||||
UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge()
|
||||
: merger_mutator.getMaxSourcePartSizeForMutation();
|
||||
/** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed),
|
||||
* then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size,
|
||||
* because it may be ordered by OPTIMIZE or early with different settings.
|
||||
* Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges,
|
||||
* because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL).
|
||||
*/
|
||||
UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge();
|
||||
if (max_source_parts_size != data.settings.max_bytes_to_merge_at_max_space_in_pool
|
||||
&& sum_parts_size_in_bytes > max_source_parts_size)
|
||||
bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data.settings.max_bytes_to_merge_at_max_space_in_pool);
|
||||
|
||||
if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size)
|
||||
{
|
||||
String reason = "Not executing log entry for part " + entry.new_part_name
|
||||
String reason = "Not executing log entry " + entry.typeToString() + " for part " + entry.new_part_name
|
||||
+ " because source parts size (" + formatReadableSizeWithBinarySuffix(sum_parts_size_in_bytes)
|
||||
+ ") is greater than the current maximum (" + formatReadableSizeWithBinarySuffix(max_source_parts_size) + ").";
|
||||
LOG_DEBUG(log, reason);
|
||||
@ -1154,17 +1158,21 @@ bool ReplicatedMergeTreeQueue::processEntry(
|
||||
}
|
||||
|
||||
|
||||
size_t ReplicatedMergeTreeQueue::countMergesAndPartMutations() const
|
||||
std::pair<size_t, size_t> ReplicatedMergeTreeQueue::countMergesAndPartMutations() const
|
||||
{
|
||||
std::lock_guard lock(state_mutex);
|
||||
|
||||
size_t count = 0;
|
||||
size_t count_merges = 0;
|
||||
size_t count_mutations = 0;
|
||||
for (const auto & entry : queue)
|
||||
if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS
|
||||
|| entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART)
|
||||
++count;
|
||||
{
|
||||
if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS)
|
||||
++count_merges;
|
||||
else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART)
|
||||
++count_mutations;
|
||||
}
|
||||
|
||||
return count;
|
||||
return std::make_pair(count_merges, count_mutations);
|
||||
}
|
||||
|
||||
|
||||
|
@ -296,7 +296,7 @@ public:
|
||||
bool processEntry(std::function<zkutil::ZooKeeperPtr()> get_zookeeper, LogEntryPtr & entry, const std::function<bool(LogEntryPtr &)> func);
|
||||
|
||||
/// Count the number of merges and mutations of single parts in the queue.
|
||||
size_t countMergesAndPartMutations() const;
|
||||
std::pair<size_t, size_t> countMergesAndPartMutations() const;
|
||||
|
||||
/// Count the total number of active mutations.
|
||||
size_t countMutations() const;
|
||||
|
@ -51,6 +51,8 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
index_granularity_bytes = data.settings.index_granularity_bytes;
|
||||
else
|
||||
index_granularity_bytes = 0;
|
||||
|
||||
constraints = data.getConstraints().toString();
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
|
||||
@ -80,6 +82,9 @@ void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
|
||||
|
||||
if (index_granularity_bytes != 0)
|
||||
out << "granularity bytes: " << index_granularity_bytes << "\n";
|
||||
|
||||
if (!constraints.empty())
|
||||
out << "constraints: " << constraints << "\n";
|
||||
}
|
||||
|
||||
String ReplicatedMergeTreeTableMetadata::toString() const
|
||||
@ -123,6 +128,9 @@ void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
|
||||
}
|
||||
else
|
||||
index_granularity_bytes = 0;
|
||||
|
||||
if (checkString("constraints: ", in))
|
||||
in >> constraints >> "\n";
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const String & s)
|
||||
@ -235,6 +243,21 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
if (constraints != from_zk.constraints)
|
||||
{
|
||||
if (allow_alter)
|
||||
{
|
||||
diff.constraints_changed = true;
|
||||
diff.new_constraints = from_zk.constraints;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in constraints."
|
||||
" Stored in ZooKeeper: " + from_zk.constraints +
|
||||
", local: " + constraints,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
if (from_zk.index_granularity_bytes_found_in_zk && index_granularity_bytes != from_zk.index_granularity_bytes)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in index granularity bytes."
|
||||
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity_bytes) +
|
||||
|
@ -26,6 +26,7 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
String partition_key;
|
||||
String sorting_key;
|
||||
String skip_indices;
|
||||
String constraints;
|
||||
String ttl_table;
|
||||
UInt64 index_granularity_bytes;
|
||||
|
||||
@ -46,10 +47,16 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
bool skip_indices_changed = false;
|
||||
String new_skip_indices;
|
||||
|
||||
bool constraints_changed = false;
|
||||
String new_constraints;
|
||||
|
||||
bool ttl_table_changed = false;
|
||||
String new_ttl_table;
|
||||
|
||||
bool empty() const { return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed; }
|
||||
bool empty() const
|
||||
{
|
||||
return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed;
|
||||
}
|
||||
};
|
||||
|
||||
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const;
|
||||
|
@ -574,6 +574,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
ASTPtr sample_by_ast;
|
||||
ASTPtr ttl_table_ast;
|
||||
IndicesDescription indices_description;
|
||||
ConstraintsDescription constraints_description;
|
||||
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
|
||||
|
||||
if (is_extended_storage_def)
|
||||
@ -602,7 +603,10 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
indices_description.indices.push_back(
|
||||
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
|
||||
|
||||
|
||||
if (args.query.columns_list && args.query.columns_list->constraints)
|
||||
for (const auto & constraint : args.query.columns_list->constraints->children)
|
||||
constraints_description.constraints.push_back(
|
||||
std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint->clone()));
|
||||
storage_settings.loadFromQuery(*args.storage_def);
|
||||
}
|
||||
else
|
||||
@ -639,14 +643,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
if (replicated)
|
||||
return StorageReplicatedMergeTree::create(
|
||||
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
|
||||
args.columns, indices_description,
|
||||
args.columns, indices_description, constraints_description,
|
||||
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
|
||||
sample_by_ast, ttl_table_ast, merging_params, storage_settings,
|
||||
args.has_force_restore_data_flag);
|
||||
else
|
||||
return StorageMergeTree::create(
|
||||
args.data_path, args.database_name, args.table_name, args.columns, indices_description,
|
||||
args.attach, args.context, date_column_name, partition_by_ast, order_by_ast,
|
||||
constraints_description, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast,
|
||||
primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings,
|
||||
args.has_force_restore_data_flag);
|
||||
}
|
||||
|
@ -23,6 +23,14 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
res.detach = command_ast->detach;
|
||||
return res;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_DETACHED_PARTITION)
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = DROP_DETACHED_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.part = command_ast->part;
|
||||
return res;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::ATTACH_PARTITION)
|
||||
{
|
||||
PartitionCommand res;
|
||||
|
@ -22,6 +22,7 @@ struct PartitionCommand
|
||||
CLEAR_COLUMN,
|
||||
CLEAR_INDEX,
|
||||
DROP_PARTITION,
|
||||
DROP_DETACHED_PARTITION,
|
||||
FETCH_PARTITION,
|
||||
FREEZE_ALL_PARTITIONS,
|
||||
FREEZE_PARTITION,
|
||||
@ -37,7 +38,7 @@ struct PartitionCommand
|
||||
/// true for DETACH PARTITION.
|
||||
bool detach = false;
|
||||
|
||||
/// true for ATTACH PART (and false for PARTITION)
|
||||
/// true for ATTACH PART and DROP DETACHED PART (and false for PARTITION)
|
||||
bool part = false;
|
||||
|
||||
/// For ATTACH PARTITION partition FROM db.table
|
||||
|
@ -701,8 +701,9 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -390,8 +390,9 @@ void StorageDistributed::alter(
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -68,14 +68,6 @@ StoragePtr StorageFactory::get(
|
||||
|
||||
name = "LiveView";
|
||||
}
|
||||
else if (query.is_live_channel)
|
||||
{
|
||||
|
||||
if (query.storage)
|
||||
throw Exception("Specifying ENGINE is not allowed for a LiveChannel", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
name = "LiveChannel";
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type.
|
||||
@ -137,12 +129,6 @@ StoragePtr StorageFactory::get(
|
||||
"Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
else if (name == "LiveChannel")
|
||||
{
|
||||
throw Exception(
|
||||
"Direct creation of tables with ENGINE LiveChannel is not supported, use CREATE LIVE CHANNEL statement",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,347 +0,0 @@
|
||||
/* Copyright (c) 2018 BlackBerry Limited
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Condition.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/BlocksBlockInputStream.h>
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/ProxyStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IAST;
|
||||
|
||||
struct BlocksMetadata
|
||||
{
|
||||
String hash;
|
||||
UInt64 version;
|
||||
};
|
||||
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using BlocksMetadataPtr = std::shared_ptr<BlocksMetadata>;
|
||||
using SipHashPtr = std::shared_ptr<SipHash>;
|
||||
|
||||
class StorageLiveView : public ext::shared_ptr_helper<StorageLiveView>, public IStorage
|
||||
{
|
||||
friend struct ext::shared_ptr_helper<StorageLiveView>;
|
||||
friend class LiveViewBlockOutputStream;
|
||||
|
||||
public:
|
||||
~StorageLiveView() override;
|
||||
String getName() const override { return "LiveView"; }
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
String getSelectDatabaseName() const { return select_database_name; }
|
||||
String getSelectTableName() const { return select_table_name; }
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
bool hasColumn(const String & column_name) const override;
|
||||
|
||||
// const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||
ASTPtr getInnerQuery() const { return inner_query->clone(); }
|
||||
|
||||
/// It is passed inside the query and solved at its level.
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
|
||||
/// Mutex for the blocks and ready condition
|
||||
Poco::FastMutex mutex;
|
||||
/// New blocks ready condition to broadcast to readers
|
||||
/// that new blocks are available
|
||||
Poco::Condition condition;
|
||||
|
||||
bool isTemporary() { return is_temporary; }
|
||||
|
||||
/// Check if we have any readers
|
||||
/// must be called with mutex locked
|
||||
bool hasUsers()
|
||||
{
|
||||
return blocks_ptr.use_count() > 1;
|
||||
}
|
||||
|
||||
/// Check we have any active readers
|
||||
/// must be called with mutex locked
|
||||
bool hasActiveUsers()
|
||||
{
|
||||
return active_ptr.use_count() > 1;
|
||||
}
|
||||
/// Background thread for temporary tables
|
||||
/// which drops this table if there are no users
|
||||
void startNoUsersThread(const UInt64 & timeout);
|
||||
Poco::FastMutex noUsersThreadMutex;
|
||||
bool noUsersThreadWakeUp{false};
|
||||
Poco::Condition noUsersThreadCondition;
|
||||
/// Get blocks hash
|
||||
/// must be called with mutex locked
|
||||
String getBlocksHashKey()
|
||||
{
|
||||
if (*blocks_metadata_ptr)
|
||||
return (*blocks_metadata_ptr)->hash;
|
||||
return "";
|
||||
}
|
||||
/// Get blocks version
|
||||
/// must be called with mutex locked
|
||||
UInt64 getBlocksVersion()
|
||||
{
|
||||
if (*blocks_metadata_ptr)
|
||||
return (*blocks_metadata_ptr)->version;
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Reset blocks
|
||||
/// must be called with mutex locked
|
||||
void reset()
|
||||
{
|
||||
(*blocks_ptr).reset();
|
||||
if (*blocks_metadata_ptr)
|
||||
(*blocks_metadata_ptr)->hash.clear();
|
||||
mergeable_blocks.reset();
|
||||
}
|
||||
|
||||
void checkTableCanBeDropped() const override;
|
||||
void drop() override;
|
||||
void startup() override;
|
||||
void shutdown() override;
|
||||
|
||||
void refresh(const Context & context);
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
BlockInputStreams watch(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
std::shared_ptr<BlocksPtr> getBlocksPtr() { return blocks_ptr; }
|
||||
BlocksPtrs getMergeableBlocks() { return mergeable_blocks; }
|
||||
void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; }
|
||||
std::shared_ptr<bool> getActivePtr() { return active_ptr; }
|
||||
|
||||
/// Read new data blocks that store query result
|
||||
bool getNewBlocks();
|
||||
|
||||
Block getHeader() const;
|
||||
|
||||
static void writeIntoLiveView(StorageLiveView & live_view,
|
||||
const Block & block,
|
||||
const Context & context,
|
||||
BlockOutputStreamPtr & output)
|
||||
{
|
||||
/// Check if live view has any readers if not
|
||||
/// just reset blocks to empty and do nothing else
|
||||
/// When first reader comes the blocks will be read.
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(live_view.mutex);
|
||||
if (!live_view.hasActiveUsers())
|
||||
{
|
||||
live_view.reset();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
bool is_block_processed = false;
|
||||
BlockInputStreams from;
|
||||
BlocksPtrs mergeable_blocks;
|
||||
BlocksPtr new_mergeable_blocks = std::make_shared<Blocks>();
|
||||
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(live_view.mutex);
|
||||
|
||||
mergeable_blocks = live_view.getMergeableBlocks();
|
||||
if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh)
|
||||
{
|
||||
mergeable_blocks = std::make_shared<std::vector<BlocksPtr>>();
|
||||
BlocksPtr base_mergeable_blocks = std::make_shared<Blocks>();
|
||||
InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names());
|
||||
auto view_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
interpreter.execute().in);
|
||||
while (Block this_block = view_mergeable_stream->read())
|
||||
base_mergeable_blocks->push_back(this_block);
|
||||
mergeable_blocks->push_back(base_mergeable_blocks);
|
||||
live_view.setMergeableBlocks(mergeable_blocks);
|
||||
|
||||
/// Create from streams
|
||||
for (auto & blocks_ : *mergeable_blocks)
|
||||
{
|
||||
if (blocks_->empty())
|
||||
continue;
|
||||
auto sample_block = blocks_->front().cloneEmpty();
|
||||
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
|
||||
from.push_back(std::move(stream));
|
||||
}
|
||||
|
||||
is_block_processed = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!is_block_processed)
|
||||
{
|
||||
auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName());
|
||||
BlockInputStreams streams = {std::make_shared<OneBlockInputStream>(block)};
|
||||
auto proxy_storage = std::make_shared<ProxyStorage>(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns);
|
||||
InterpreterSelectQuery select_block(live_view.getInnerQuery(),
|
||||
context, proxy_storage,
|
||||
QueryProcessingStage::WithMergeableState);
|
||||
auto data_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
select_block.execute().in);
|
||||
while (Block this_block = data_mergeable_stream->read())
|
||||
new_mergeable_blocks->push_back(this_block);
|
||||
|
||||
if (new_mergeable_blocks->empty())
|
||||
return;
|
||||
|
||||
{
|
||||
Poco::FastMutex::ScopedLock lock(live_view.mutex);
|
||||
|
||||
mergeable_blocks = live_view.getMergeableBlocks();
|
||||
mergeable_blocks->push_back(new_mergeable_blocks);
|
||||
|
||||
/// Create from streams
|
||||
for (auto & blocks_ : *mergeable_blocks)
|
||||
{
|
||||
if (blocks_->empty())
|
||||
continue;
|
||||
auto sample_block = blocks_->front().cloneEmpty();
|
||||
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
|
||||
from.push_back(std::move(stream));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName());
|
||||
auto proxy_storage = std::make_shared<ProxyStorage>(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState);
|
||||
InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete);
|
||||
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
||||
|
||||
/// Squashing is needed here because the 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).
|
||||
data = std::make_shared<SquashingBlockInputStream>(
|
||||
data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes);
|
||||
|
||||
copyData(*data, *output);
|
||||
}
|
||||
|
||||
private:
|
||||
String select_database_name;
|
||||
String select_table_name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
ASTPtr inner_query;
|
||||
Context & global_context;
|
||||
bool is_temporary {false};
|
||||
mutable Block sample_block;
|
||||
|
||||
/// Active users
|
||||
std::shared_ptr<bool> active_ptr;
|
||||
/// Current data blocks that store query result
|
||||
std::shared_ptr<BlocksPtr> blocks_ptr;
|
||||
/// Current data blocks metadata
|
||||
std::shared_ptr<BlocksMetadataPtr> blocks_metadata_ptr;
|
||||
BlocksPtrs mergeable_blocks;
|
||||
|
||||
void noUsersThread(const UInt64 & timeout);
|
||||
std::thread no_users_thread;
|
||||
std::atomic<bool> shutdown_called{false};
|
||||
std::atomic<bool> startnousersthread_called{false};
|
||||
UInt64 temporary_live_view_timeout;
|
||||
|
||||
StorageLiveView(
|
||||
const String & table_name_,
|
||||
const String & database_name_,
|
||||
Context & local_context,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns
|
||||
);
|
||||
};
|
||||
|
||||
class LiveViewBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
new_blocks = std::make_shared<Blocks>();
|
||||
new_blocks_metadata = std::make_shared<BlocksMetadata>();
|
||||
new_hash = std::make_shared<SipHash>();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
UInt128 key;
|
||||
String key_str;
|
||||
|
||||
new_hash->get128(key.low, key.high);
|
||||
key_str = key.toHexString();
|
||||
|
||||
Poco::FastMutex::ScopedLock lock(storage.mutex);
|
||||
|
||||
if (storage.getBlocksHashKey() != key_str)
|
||||
{
|
||||
new_blocks_metadata->hash = key_str;
|
||||
new_blocks_metadata->version = storage.getBlocksVersion() + 1;
|
||||
|
||||
for (auto & block : *new_blocks)
|
||||
{
|
||||
block.insert({DataTypeUInt64().createColumnConst(
|
||||
block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(),
|
||||
std::make_shared<DataTypeUInt64>(),
|
||||
"_version"});
|
||||
}
|
||||
|
||||
(*storage.blocks_ptr) = new_blocks;
|
||||
(*storage.blocks_metadata_ptr) = new_blocks_metadata;
|
||||
|
||||
storage.condition.broadcast();
|
||||
}
|
||||
|
||||
new_blocks.reset();
|
||||
new_blocks_metadata.reset();
|
||||
new_hash.reset();
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
new_blocks->push_back(block);
|
||||
block.updateHash(*new_hash);
|
||||
}
|
||||
|
||||
Block getHeader() const override { return storage.getHeader(); }
|
||||
|
||||
private:
|
||||
BlocksPtr new_blocks;
|
||||
BlocksMetadataPtr new_blocks_metadata;
|
||||
SipHashPtr new_hash;
|
||||
StorageLiveView & storage;
|
||||
};
|
||||
|
||||
}
|
@ -403,8 +403,9 @@ void StorageMerge::alter(
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(new_columns);
|
||||
}
|
||||
|
||||
|
@ -51,6 +51,7 @@ StorageMergeTree::StorageMergeTree(
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
bool attach,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
@ -64,7 +65,7 @@ StorageMergeTree::StorageMergeTree(
|
||||
bool has_force_restore_data_flag)
|
||||
: MergeTreeData(database_name_, table_name_,
|
||||
path_ + escapeForFileName(table_name_) + '/',
|
||||
columns_, indices_,
|
||||
columns_, indices_, constraints_,
|
||||
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
|
||||
sample_by_ast_, ttl_table_ast_, merging_params_,
|
||||
settings_, false, attach),
|
||||
@ -252,8 +253,9 @@ void StorageMergeTree::alter(
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
return;
|
||||
}
|
||||
@ -262,15 +264,14 @@ void StorageMergeTree::alter(
|
||||
auto merge_blocker = merger_mutator.merges_blocker.cancel();
|
||||
|
||||
lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
checkAlter(params, context);
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
auto transactions = prepareAlterTransactions(new_columns, new_indices, context);
|
||||
|
||||
@ -289,11 +290,10 @@ void StorageMergeTree::alter(
|
||||
if (new_ttl_table_ast.get() != ttl_table_ast.get())
|
||||
storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast);
|
||||
};
|
||||
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
|
||||
setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints);
|
||||
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
|
||||
|
||||
@ -630,8 +630,6 @@ bool StorageMergeTree::tryMutatePart()
|
||||
/// You must call destructor with unlocked `currently_merging_mutex`.
|
||||
std::optional<CurrentlyMergingPartsTagger> tagger;
|
||||
{
|
||||
auto disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
|
||||
std::lock_guard lock(currently_merging_mutex);
|
||||
|
||||
if (current_mutations_by_version.empty())
|
||||
@ -647,8 +645,7 @@ bool StorageMergeTree::tryMutatePart()
|
||||
if (mutations_begin_it == mutations_end_it)
|
||||
continue;
|
||||
|
||||
auto estimated_needed_space = MergeTreeDataMergerMutator::estimateNeededDiskSpace({part});
|
||||
if (estimated_needed_space > disk_space)
|
||||
if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk)
|
||||
continue;
|
||||
|
||||
for (auto it = mutations_begin_it; it != mutations_end_it; ++it)
|
||||
@ -661,7 +658,7 @@ bool StorageMergeTree::tryMutatePart()
|
||||
future_part.part_info = new_part_info;
|
||||
future_part.name = part->getNewName(new_part_info);
|
||||
|
||||
tagger.emplace(future_part, estimated_needed_space, *this);
|
||||
tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this);
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -848,10 +845,11 @@ void StorageMergeTree::clearColumnOrIndexInPartition(const ASTPtr & partition, c
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
ASTPtr ignored_ttl_table_ast;
|
||||
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
|
||||
auto columns_for_parts = new_columns.getAllPhysical();
|
||||
for (const auto & part : parts)
|
||||
@ -950,6 +948,10 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma
|
||||
dropPartition(command.partition, command.detach, context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::DROP_DETACHED_PARTITION:
|
||||
dropDetached(command.partition, command.part, context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::ATTACH_PARTITION:
|
||||
attachPartition(command.partition, command.part, context);
|
||||
break;
|
||||
@ -1014,6 +1016,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons
|
||||
|
||||
/// TODO: should we include PreComitted parts like in Replicated case?
|
||||
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
|
||||
// TODO should we throw an exception if parts_to_remove is empty?
|
||||
removePartsFromWorkingSet(parts_to_remove, true);
|
||||
|
||||
if (detach)
|
||||
@ -1037,51 +1040,14 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par
|
||||
{
|
||||
// TODO: should get some locks to prevent race with 'alter … modify column'
|
||||
|
||||
String partition_id;
|
||||
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
|
||||
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, context, renamed_parts);
|
||||
|
||||
if (attach_part)
|
||||
partition_id = partition->as<ASTLiteral &>().value.safeGet<String>();
|
||||
else
|
||||
partition_id = getPartitionIDFromQuery(partition, context);
|
||||
|
||||
String source_dir = "detached/";
|
||||
|
||||
/// Let's make a list of parts to add.
|
||||
Strings parts;
|
||||
if (attach_part)
|
||||
for (size_t i = 0; i < loaded_parts.size(); ++i)
|
||||
{
|
||||
parts.push_back(partition_id);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
|
||||
ActiveDataPartSet active_parts(format_version);
|
||||
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
|
||||
{
|
||||
const String & name = it.name();
|
||||
MergeTreePartInfo part_info;
|
||||
if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)
|
||||
|| part_info.partition_id != partition_id)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
LOG_DEBUG(log, "Found part " << name);
|
||||
active_parts.add(name);
|
||||
}
|
||||
LOG_DEBUG(log, active_parts.size() << " of them are active");
|
||||
parts = active_parts.getParts();
|
||||
}
|
||||
|
||||
for (const auto & source_part_name : parts)
|
||||
{
|
||||
String source_path = source_dir + source_part_name;
|
||||
|
||||
LOG_DEBUG(log, "Checking data");
|
||||
MutableDataPartPtr part = loadPartAndFixMetadata(source_path);
|
||||
|
||||
LOG_INFO(log, "Attaching part " << source_part_name << " from " << source_path);
|
||||
renameTempPartAndAdd(part, &increment);
|
||||
|
||||
LOG_INFO(log, "Attaching part " << loaded_parts[i]->name << " from " << renamed_parts.old_and_new_names[i].second);
|
||||
renameTempPartAndAdd(loaded_parts[i], &increment);
|
||||
renamed_parts.old_and_new_names[i].first.clear();
|
||||
LOG_INFO(log, "Finished attaching part");
|
||||
}
|
||||
|
||||
@ -1161,6 +1127,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
|
||||
{
|
||||
if (action_type == ActionLocks::PartsMerge)
|
||||
|
@ -142,6 +142,7 @@ protected:
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
bool attach,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
|
@ -38,8 +38,9 @@ void StorageNull::alter(
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
IndicesDescription new_indices = getIndices();
|
||||
ConstraintsDescription new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -193,6 +193,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
@ -205,7 +206,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
bool has_force_restore_data_flag)
|
||||
: MergeTreeData(database_name_, table_name_,
|
||||
path_ + escapeForFileName(table_name_) + '/',
|
||||
columns_, indices_,
|
||||
columns_, indices_, constraints_,
|
||||
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
|
||||
sample_by_ast_, ttl_table_ast_, merging_params_,
|
||||
settings_, true, attach,
|
||||
@ -426,6 +427,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
IDatabase::ASTModifier storage_modifier;
|
||||
if (!metadata_diff.empty())
|
||||
@ -455,6 +457,9 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
if (metadata_diff.skip_indices_changed)
|
||||
new_indices = IndicesDescription::parse(metadata_diff.new_skip_indices);
|
||||
|
||||
if (metadata_diff.constraints_changed)
|
||||
new_constraints = ConstraintsDescription::parse(metadata_diff.new_constraints);
|
||||
|
||||
if (metadata_diff.ttl_table_changed)
|
||||
{
|
||||
ParserExpression parser;
|
||||
@ -480,11 +485,11 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
};
|
||||
}
|
||||
|
||||
global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, storage_modifier);
|
||||
global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
|
||||
setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints);
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
|
||||
}
|
||||
|
||||
@ -1519,10 +1524,11 @@ void StorageReplicatedMergeTree::executeClearColumnOrIndexInPartition(const LogE
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
ASTPtr ignored_ttl_table_ast;
|
||||
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
|
||||
size_t modified_parts = 0;
|
||||
auto parts = getDataParts();
|
||||
@ -2219,17 +2225,18 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
||||
/// If many merges is already queued, then will queue only small enough merges.
|
||||
/// Otherwise merge queue could be filled with only large merges,
|
||||
/// and in the same time, many small parts could be created and won't be merged.
|
||||
size_t merges_and_mutations_queued = queue.countMergesAndPartMutations();
|
||||
if (merges_and_mutations_queued >= settings.max_replicated_merges_in_queue)
|
||||
auto merges_and_mutations_queued = queue.countMergesAndPartMutations();
|
||||
size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second;
|
||||
if (merges_and_mutations_sum >= settings.max_replicated_merges_in_queue)
|
||||
{
|
||||
LOG_TRACE(log, "Number of queued merges and part mutations (" << merges_and_mutations_queued
|
||||
<< ") is greater than max_replicated_merges_in_queue ("
|
||||
LOG_TRACE(log, "Number of queued merges (" << merges_and_mutations_queued.first << ") and part mutations ("
|
||||
<< merges_and_mutations_queued.second << ") is greater than max_replicated_merges_in_queue ("
|
||||
<< settings.max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge(
|
||||
settings.max_replicated_merges_in_queue, merges_and_mutations_queued);
|
||||
settings.max_replicated_merges_in_queue, merges_and_mutations_sum);
|
||||
UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation();
|
||||
|
||||
FutureMergedMutatedPart future_merged_part;
|
||||
@ -2239,7 +2246,9 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
||||
success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts,
|
||||
future_merged_part.name, deduplicate, force_ttl);
|
||||
}
|
||||
else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0)
|
||||
/// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts
|
||||
else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0
|
||||
&& merges_and_mutations_queued.second < settings.max_replicated_mutations_in_queue)
|
||||
{
|
||||
/// Choose a part to mutate.
|
||||
DataPartsVector data_parts = getDataPartsVector();
|
||||
@ -3169,10 +3178,11 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
IndicesDescription new_indices = getIndices();
|
||||
ConstraintsDescription new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
String new_columns_str = new_columns.toString();
|
||||
if (new_columns_str != getColumns().toString())
|
||||
@ -3189,6 +3199,10 @@ void StorageReplicatedMergeTree::alter(
|
||||
if (new_indices_str != getIndices().toString())
|
||||
new_metadata.skip_indices = new_indices_str;
|
||||
|
||||
String new_constraints_str = new_constraints.toString();
|
||||
if (new_constraints_str != getConstraints().toString())
|
||||
new_metadata.constraints = new_constraints_str;
|
||||
|
||||
String new_metadata_str = new_metadata.toString();
|
||||
if (new_metadata_str != ReplicatedMergeTreeTableMetadata(*this).toString())
|
||||
changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str);
|
||||
@ -3405,6 +3419,10 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part
|
||||
dropPartition(query, command.partition, command.detach, query_context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::DROP_DETACHED_PARTITION:
|
||||
dropDetached(command.partition, command.part, query_context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::ATTACH_PARTITION:
|
||||
attachPartition(command.partition, command.part, query_context);
|
||||
break;
|
||||
@ -3604,66 +3622,16 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool
|
||||
|
||||
assertNotReadonly();
|
||||
|
||||
String partition_id;
|
||||
|
||||
if (attach_part)
|
||||
partition_id = partition->as<ASTLiteral &>().value.safeGet<String>();
|
||||
else
|
||||
partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
|
||||
String source_dir = "detached/";
|
||||
|
||||
/// Let's compose a list of parts that should be added.
|
||||
Strings parts;
|
||||
if (attach_part)
|
||||
{
|
||||
parts.push_back(partition_id);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
|
||||
ActiveDataPartSet active_parts(format_version);
|
||||
|
||||
std::set<String> part_names;
|
||||
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
|
||||
{
|
||||
String name = it.name();
|
||||
MergeTreePartInfo part_info;
|
||||
if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version))
|
||||
continue;
|
||||
if (part_info.partition_id != partition_id)
|
||||
continue;
|
||||
LOG_DEBUG(log, "Found part " << name);
|
||||
active_parts.add(name);
|
||||
part_names.insert(name);
|
||||
}
|
||||
LOG_DEBUG(log, active_parts.size() << " of them are active");
|
||||
parts = active_parts.getParts();
|
||||
|
||||
/// Inactive parts rename so they can not be attached in case of repeated ATTACH.
|
||||
for (const auto & name : part_names)
|
||||
{
|
||||
String containing_part = active_parts.getContainingPart(name);
|
||||
if (!containing_part.empty() && containing_part != name)
|
||||
Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name);
|
||||
}
|
||||
}
|
||||
|
||||
/// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
|
||||
LOG_DEBUG(log, "Checking parts");
|
||||
std::vector<MutableDataPartPtr> loaded_parts;
|
||||
for (const String & part : parts)
|
||||
{
|
||||
LOG_DEBUG(log, "Checking part " << part);
|
||||
loaded_parts.push_back(loadPartAndFixMetadata(source_dir + part));
|
||||
}
|
||||
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
|
||||
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts);
|
||||
|
||||
ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here.
|
||||
for (auto & part : loaded_parts)
|
||||
for (size_t i = 0; i < loaded_parts.size(); ++i)
|
||||
{
|
||||
String old_name = part->name;
|
||||
output.writeExistingPart(part);
|
||||
LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name);
|
||||
String old_name = loaded_parts[i]->name;
|
||||
output.writeExistingPart(loaded_parts[i]);
|
||||
renamed_parts.old_and_new_names[i].first.clear();
|
||||
LOG_DEBUG(log, "Attached part " << old_name << " as " << loaded_parts[i]->name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -531,6 +531,7 @@ protected:
|
||||
const String & path_, const String & database_name_, const String & name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
@ -31,12 +32,12 @@ protected:
|
||||
setColumns(ColumnsDescription{{
|
||||
{"database", std::make_shared<DataTypeString>()},
|
||||
{"table", std::make_shared<DataTypeString>()},
|
||||
{"partition_id", std::make_shared<DataTypeString>()},
|
||||
{"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"reason", std::make_shared<DataTypeString>()},
|
||||
{"min_block_number", std::make_shared<DataTypeInt64>()},
|
||||
{"max_block_number", std::make_shared<DataTypeInt64>()},
|
||||
{"level", std::make_shared<DataTypeUInt32>()}
|
||||
{"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
|
||||
{"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
|
||||
{"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt32>())}
|
||||
}});
|
||||
}
|
||||
|
||||
@ -62,12 +63,12 @@ protected:
|
||||
size_t i = 0;
|
||||
new_columns[i++]->insert(info.database);
|
||||
new_columns[i++]->insert(info.table);
|
||||
new_columns[i++]->insert(p.partition_id);
|
||||
new_columns[i++]->insert(p.getPartName());
|
||||
new_columns[i++]->insert(p.prefix);
|
||||
new_columns[i++]->insert(p.min_block);
|
||||
new_columns[i++]->insert(p.max_block);
|
||||
new_columns[i++]->insert(p.level);
|
||||
new_columns[i++]->insert(p.valid_name ? p.partition_id : Field());
|
||||
new_columns[i++]->insert(p.dir_name);
|
||||
new_columns[i++]->insert(p.valid_name ? p.prefix : Field());
|
||||
new_columns[i++]->insert(p.valid_name ? p.min_block : Field());
|
||||
new_columns[i++]->insert(p.valid_name ? p.max_block : Field());
|
||||
new_columns[i++]->insert(p.valid_name ? p.level : Field());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -25,7 +25,6 @@ void registerStorageJoin(StorageFactory & factory);
|
||||
void registerStorageView(StorageFactory & factory);
|
||||
void registerStorageMaterializedView(StorageFactory & factory);
|
||||
void registerStorageLiveView(StorageFactory & factory);
|
||||
//void registerStorageLiveChannel(StorageFactory & factory);
|
||||
|
||||
#if USE_HDFS
|
||||
void registerStorageHDFS(StorageFactory & factory);
|
||||
@ -67,7 +66,6 @@ void registerStorages()
|
||||
registerStorageView(factory);
|
||||
registerStorageMaterializedView(factory);
|
||||
registerStorageLiveView(factory);
|
||||
//registerStorageLiveChannel(factory);
|
||||
|
||||
#if USE_HDFS
|
||||
registerStorageHDFS(factory);
|
||||
|
244
dbms/tests/integration/test_partition/test.py
Normal file
244
dbms/tests/integration/test_partition/test.py
Normal file
@ -0,0 +1,244 @@
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance')
|
||||
q = instance.query
|
||||
path_to_data = '/var/lib/clickhouse/'
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
q('CREATE DATABASE test')
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def partition_table_simple(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.partition")
|
||||
q("CREATE TABLE test.partition (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) "
|
||||
"ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) "
|
||||
"SETTINGS index_granularity=8192, index_granularity_bytes=0")
|
||||
q("INSERT INTO test.partition ( x ) VALUES ( now() )")
|
||||
q("INSERT INTO test.partition ( x ) VALUES ( now()+1 )")
|
||||
|
||||
yield
|
||||
|
||||
q('DROP TABLE test.partition')
|
||||
|
||||
|
||||
def test_partition_simple(partition_table_simple):
|
||||
q("ALTER TABLE test.partition DETACH PARTITION 197001")
|
||||
q("ALTER TABLE test.partition ATTACH PARTITION 197001")
|
||||
q("OPTIMIZE TABLE test.partition")
|
||||
|
||||
|
||||
def exec_bash(cmd):
|
||||
cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"'))
|
||||
return instance.exec_in_container(cmd)
|
||||
|
||||
|
||||
def partition_complex_assert_columns_txt():
|
||||
path_to_parts = path_to_data + 'data/test/partition/'
|
||||
parts = TSV(q("SELECT name FROM system.parts WHERE database='test' AND table='partition'"))
|
||||
for part_name in parts.lines:
|
||||
path_to_columns = path_to_parts + part_name + '/columns.txt'
|
||||
# 2 header lines + 3 columns
|
||||
assert exec_bash('cat {} | wc -l'.format(path_to_columns)) == u'5\n'
|
||||
|
||||
|
||||
def partition_complex_assert_checksums():
|
||||
# Do `cd` for consistent output for reference
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cmd = 'cd ' + path_to_data + " && find shadow -type f -exec md5sum {} \\;" \
|
||||
" | grep partition" \
|
||||
" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g'" \
|
||||
" | sort" \
|
||||
" | uniq"
|
||||
|
||||
checksums = "082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700102_2_2_0/k.bin\n" \
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700201_1_1_0/v1.bin\n" \
|
||||
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition/19700102_2_2_0/minmax_p.idx\n" \
|
||||
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition/19700102_2_2_0/partition.dat\n" \
|
||||
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition/19700201_1_1_0/partition.dat\n" \
|
||||
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition/19700102_2_2_0/checksums.txt\n" \
|
||||
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition/19700102_2_2_0/v1.bin\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/k.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/p.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/v1.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/k.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/p.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/v1.mrk\n" \
|
||||
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition/19700201_1_1_0/primary.idx\n" \
|
||||
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition/19700201_1_1_0/checksums.txt\n" \
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700102_2_2_0/columns.txt\n" \
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0/columns.txt\n" \
|
||||
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0/p.bin\n" \
|
||||
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0/primary.idx\n" \
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0/count.txt\n" \
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0/count.txt\n" \
|
||||
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0/p.bin\n" \
|
||||
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition/19700201_1_1_0/k.bin\n" \
|
||||
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition/19700201_1_1_0/minmax_p.idx\n"
|
||||
|
||||
assert TSV(exec_bash(cmd).replace(' ', '\t')) == TSV(checksums)
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def partition_table_complex(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.partition")
|
||||
q("CREATE TABLE test.partition (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) "
|
||||
"ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0")
|
||||
q("INSERT INTO test.partition (p, k) VALUES(toDate(31), 1)")
|
||||
q("INSERT INTO test.partition (p, k) VALUES(toDate(1), 2)")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.partition")
|
||||
|
||||
|
||||
def test_partition_complex(partition_table_complex):
|
||||
|
||||
partition_complex_assert_columns_txt()
|
||||
|
||||
q("ALTER TABLE test.partition FREEZE")
|
||||
|
||||
partition_complex_assert_checksums()
|
||||
|
||||
q("ALTER TABLE test.partition DETACH PARTITION 197001")
|
||||
q("ALTER TABLE test.partition ATTACH PARTITION 197001")
|
||||
|
||||
partition_complex_assert_columns_txt()
|
||||
|
||||
q("ALTER TABLE test.partition MODIFY COLUMN v1 Int8")
|
||||
|
||||
# Check the backup hasn't changed
|
||||
partition_complex_assert_checksums()
|
||||
|
||||
q("OPTIMIZE TABLE test.partition")
|
||||
|
||||
expected = TSV('31\t1\t2\n'
|
||||
'1\t2\t3')
|
||||
res = q("SELECT toUInt16(p), k, v1 FROM test.partition ORDER BY k")
|
||||
assert(TSV(res) == expected)
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def cannot_attach_active_part_table(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.attach_active")
|
||||
q("CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n")
|
||||
q("INSERT INTO test.attach_active SELECT number FROM system.numbers LIMIT 16")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.attach_active")
|
||||
|
||||
|
||||
def test_cannot_attach_active_part(cannot_attach_active_part_table):
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.attach_active ATTACH PART '../1_2_2_0'")
|
||||
print error
|
||||
assert 0 <= error.find('Invalid part name')
|
||||
|
||||
res = q("SElECT name FROM system.parts WHERE table='attach_active' AND database='test' ORDER BY name")
|
||||
assert TSV(res) == TSV('0_1_1_0\n1_2_2_0\n2_3_3_0\n3_4_4_0')
|
||||
assert TSV(q("SElECT count(), sum(n) FROM test.attach_active")) == TSV('16\t120')
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def attach_check_all_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.attach_partition")
|
||||
q("CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n")
|
||||
q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8")
|
||||
q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.attach_partition")
|
||||
q("SYSTEM START MERGES")
|
||||
|
||||
|
||||
def test_attach_check_all_parts(attach_check_all_parts_table):
|
||||
q("ALTER TABLE test.attach_partition DETACH PARTITION 0")
|
||||
|
||||
path_to_detached = path_to_data + 'data/test/attach_partition/detached/'
|
||||
exec_bash('mkdir {}'.format(path_to_detached + '0_5_5_0'))
|
||||
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_1_1_0', path_to_detached + 'attaching_0_6_6_0'))
|
||||
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0', path_to_detached + 'deleting_0_7_7_0'))
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.attach_partition ATTACH PARTITION 0")
|
||||
assert 0 <= error.find('No columns in part 0_5_5_0')
|
||||
|
||||
parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(parts) == TSV('1_2_2_0\n1_4_4_0')
|
||||
detached = q("SELECT name FROM system.detached_parts "
|
||||
"WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_1_1_0\n0_3_3_0\n0_5_5_0\nattaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
||||
exec_bash('rm -r {}'.format(path_to_detached + '0_5_5_0'))
|
||||
|
||||
q("ALTER TABLE test.attach_partition ATTACH PARTITION 0")
|
||||
parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
expected = '0_5_5_0\n0_6_6_0\n1_2_2_0\n1_4_4_0'
|
||||
assert TSV(parts) == TSV(expected)
|
||||
assert TSV(q("SElECT count(), sum(n) FROM test.attach_partition")) == TSV('16\t120')
|
||||
|
||||
detached = q("SELECT name FROM system.detached_parts "
|
||||
"WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('attaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def drop_detached_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.drop_detached")
|
||||
q("CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n")
|
||||
q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8")
|
||||
q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.drop_detached")
|
||||
q("SYSTEM START MERGES")
|
||||
|
||||
|
||||
def test_drop_detached_parts(drop_detached_parts_table):
|
||||
s = {"allow_drop_detached": 1}
|
||||
q("ALTER TABLE test.drop_detached DETACH PARTITION 0")
|
||||
q("ALTER TABLE test.drop_detached DETACH PARTITION 1")
|
||||
|
||||
path_to_detached = path_to_data + 'data/test/drop_detached/detached/'
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'attaching_0_6_6_0'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'deleting_0_7_7_0'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'any_other_name'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'prefix_1_2_2_0_0'))
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART '../1_2_2_0'", settings=s)
|
||||
assert 0 <= error.find('Invalid part name')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PART '0_1_1_0'", settings=s)
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'attaching_0_6_6_0'", settings=s)
|
||||
assert 0 <= error.find('Cannot drop part')
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'deleting_0_7_7_0'", settings=s)
|
||||
assert 0 <= error.find('Cannot drop part')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PART 'any_other_name'", settings=s)
|
||||
|
||||
detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_3_3_0\n1_2_2_0\n1_4_4_0\nattaching_0_6_6_0\ndeleting_0_7_7_0\nprefix_1_2_2_0_0')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PARTITION 1", settings=s)
|
||||
detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_3_3_0\nattaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
@ -0,0 +1,6 @@
|
||||
<yandex>
|
||||
<merge_tree>
|
||||
<parts_to_delay_insert>50</parts_to_delay_insert>
|
||||
<parts_to_throw_insert>50</parts_to_throw_insert>
|
||||
</merge_tree>
|
||||
</yandex>
|
@ -10,21 +10,29 @@ from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1', with_zookeeper=True)
|
||||
node1 = cluster.add_instance('node1', macros={'cluster': 'test1'}, with_zookeeper=True)
|
||||
# Check, that limits on max part size for merges doesn`t affect mutations
|
||||
node2 = cluster.add_instance('node2', main_configs=["configs/merge_tree.xml"], with_zookeeper=True)
|
||||
nodes = [node1, node2]
|
||||
node2 = cluster.add_instance('node2', macros={'cluster': 'test1'}, main_configs=["configs/merge_tree.xml"], with_zookeeper=True)
|
||||
|
||||
node3 = cluster.add_instance('node3', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_max_parts.xml"], with_zookeeper=True)
|
||||
node4 = cluster.add_instance('node4', macros={'cluster': 'test2'}, main_configs=["configs/merge_tree_max_parts.xml"], with_zookeeper=True)
|
||||
|
||||
node5 = cluster.add_instance('node5', macros={'cluster': 'test3'}, main_configs=["configs/merge_tree_max_parts.xml"])
|
||||
|
||||
all_nodes = [node1, node2, node3, node4, node5]
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
for node in nodes:
|
||||
for node in all_nodes:
|
||||
node.query("DROP TABLE IF EXISTS test_mutations")
|
||||
|
||||
for node in nodes:
|
||||
node.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/test_mutations', '{instance}') ORDER BY x PARTITION BY toYYYYMM(d)")
|
||||
for node in [node1, node2, node3, node4]:
|
||||
node.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/test_mutations', '{instance}') ORDER BY x PARTITION BY toYYYYMM(d)")
|
||||
|
||||
node5.query("CREATE TABLE test_mutations(d Date, x UInt32, i UInt32) ENGINE MergeTree() ORDER BY x PARTITION BY toYYYYMM(d)")
|
||||
|
||||
yield cluster
|
||||
|
||||
@ -33,7 +41,8 @@ def started_cluster():
|
||||
|
||||
|
||||
class Runner:
|
||||
def __init__(self):
|
||||
def __init__(self, nodes):
|
||||
self.nodes = nodes
|
||||
self.mtx = threading.Lock()
|
||||
self.total_inserted_xs = 0
|
||||
self.total_inserted_rows = 0
|
||||
@ -49,7 +58,9 @@ class Runner:
|
||||
|
||||
self.stop_ev = threading.Event()
|
||||
|
||||
def do_insert(self, thread_num):
|
||||
self.exceptions = []
|
||||
|
||||
def do_insert(self, thread_num, partitions_num):
|
||||
self.stop_ev.wait(random.random())
|
||||
|
||||
# Each thread inserts a small random number of rows with random year, month 01 and day determined
|
||||
@ -67,7 +78,7 @@ class Runner:
|
||||
for x in xs:
|
||||
self.currently_inserting_xs[x] += 1
|
||||
|
||||
year = 2000 + random.randint(0, 10)
|
||||
year = 2000 + random.randint(0, partitions_num)
|
||||
date_str = '{year}-{month}-{day}'.format(year=year, month=month, day=day)
|
||||
payload = ''
|
||||
for x in xs:
|
||||
@ -76,7 +87,7 @@ class Runner:
|
||||
|
||||
try:
|
||||
print 'thread {}: insert for {}: {}'.format(thread_num, date_str, ','.join(str(x) for x in xs))
|
||||
random.choice(nodes).query("INSERT INTO test_mutations FORMAT TSV", payload)
|
||||
random.choice(self.nodes).query("INSERT INTO test_mutations FORMAT TSV", payload)
|
||||
|
||||
with self.mtx:
|
||||
for x in xs:
|
||||
@ -86,6 +97,7 @@ class Runner:
|
||||
|
||||
except Exception, e:
|
||||
print 'Exception while inserting,', e
|
||||
self.exceptions.append(e)
|
||||
finally:
|
||||
with self.mtx:
|
||||
for x in xs:
|
||||
@ -113,7 +125,7 @@ class Runner:
|
||||
|
||||
try:
|
||||
print 'thread {}: delete {} * {}'.format(thread_num, to_delete_count, x)
|
||||
random.choice(nodes).query("ALTER TABLE test_mutations DELETE WHERE x = {}".format(x))
|
||||
random.choice(self.nodes).query("ALTER TABLE test_mutations DELETE WHERE x = {}".format(x))
|
||||
|
||||
with self.mtx:
|
||||
self.total_mutations += 1
|
||||
@ -130,14 +142,27 @@ class Runner:
|
||||
self.stop_ev.wait(1.0 + random.random() * 2)
|
||||
|
||||
|
||||
def wait_for_mutations(nodes, number_of_mutations):
|
||||
for i in range(100): # wait for replication 80 seconds max
|
||||
time.sleep(0.8)
|
||||
|
||||
def get_done_mutations(node):
|
||||
return int(node.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations'").rstrip())
|
||||
|
||||
if all([get_done_mutations(n) == number_of_mutations for n in nodes]):
|
||||
return True
|
||||
return False
|
||||
|
||||
|
||||
def test_mutations(started_cluster):
|
||||
DURATION_SECONDS = 30
|
||||
nodes = [node1, node2]
|
||||
|
||||
runner = Runner()
|
||||
runner = Runner(nodes)
|
||||
|
||||
threads = []
|
||||
for thread_num in range(5):
|
||||
threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, )))
|
||||
threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, 10)))
|
||||
|
||||
for thread_num in (11, 12, 13):
|
||||
threads.append(threading.Thread(target=runner.do_delete, args=(thread_num,)))
|
||||
@ -155,18 +180,11 @@ def test_mutations(started_cluster):
|
||||
assert runner.total_inserted_rows > 0
|
||||
assert runner.total_mutations > 0
|
||||
|
||||
all_done = False
|
||||
for i in range(100): # wait for replication 80 seconds max
|
||||
time.sleep(0.8)
|
||||
all_done = wait_for_mutations(nodes, runner.total_mutations)
|
||||
|
||||
def get_done_mutations(node):
|
||||
return int(node.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations'").rstrip())
|
||||
|
||||
if all([get_done_mutations(n) == runner.total_mutations for n in nodes]):
|
||||
all_done = True
|
||||
break
|
||||
|
||||
print node1.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames")
|
||||
print "Total mutations: ", runner.total_mutations
|
||||
for node in nodes:
|
||||
print node.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames")
|
||||
assert all_done
|
||||
|
||||
expected_sum = runner.total_inserted_xs - runner.total_deleted_xs
|
||||
@ -174,3 +192,44 @@ def test_mutations(started_cluster):
|
||||
for i, node in enumerate(nodes):
|
||||
actual_sums.append(int(node.query("SELECT sum(x) FROM test_mutations").rstrip()))
|
||||
assert actual_sums[i] == expected_sum
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
('nodes', ),
|
||||
[
|
||||
([node5, ], ), # MergeTree
|
||||
([node3, node4], ), # ReplicatedMergeTree
|
||||
]
|
||||
)
|
||||
def test_mutations_dont_prevent_merges(started_cluster, nodes):
|
||||
for year in range(2000, 2016):
|
||||
rows = ''
|
||||
date_str = '{}-01-{}'.format(year, random.randint(1, 10))
|
||||
for i in range(10):
|
||||
rows += '{} {} {}\n'.format(date_str, random.randint(1, 10), i)
|
||||
nodes[0].query("INSERT INTO test_mutations FORMAT TSV", rows)
|
||||
|
||||
# will run mutations of 16 parts in parallel, mutations will sleep for about 20 seconds
|
||||
nodes[0].query("ALTER TABLE test_mutations UPDATE i = sleepEachRow(2) WHERE 1")
|
||||
|
||||
runner = Runner(nodes)
|
||||
threads = []
|
||||
for thread_num in range(2):
|
||||
threads.append(threading.Thread(target=runner.do_insert, args=(thread_num, 0)))
|
||||
|
||||
# will insert approx 8-10 new parts per 1 second into one partition
|
||||
for t in threads:
|
||||
t.start()
|
||||
|
||||
all_done = wait_for_mutations(nodes, 1)
|
||||
|
||||
runner.stop_ev.set()
|
||||
for t in threads:
|
||||
t.join()
|
||||
|
||||
for node in nodes:
|
||||
print node.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames")
|
||||
print node.query("SELECT partition, count(name), sum(active), sum(active*rows) FROM system.parts WHERE table ='test_mutations' GROUP BY partition FORMAT TSVWithNames")
|
||||
|
||||
assert all_done
|
||||
assert all([str(e).find("Too many parts") < 0 for e in runner.exceptions])
|
||||
|
@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
set -o errexit
|
||||
set -o pipefail
|
||||
|
||||
for i in {1..10}; do seq 1 10 | sed 's/.*/SELECT 1 % ((number + 500) % 1000) FROM system.numbers_mt LIMIT 1000;/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 --max_block_size=1 >/dev/null 2>&1 && echo 'Fail!' && break; echo -n '.'; done; echo
|
||||
for i in {1..10}; do seq 1 10 | sed 's/.*/SELECT 1 % ((number + 500) % 1000) FROM numbers_mt(1000);/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 --max_block_size=1 >/dev/null 2>&1 && echo 'Fail!' && break; echo -n '.'; done; echo
|
||||
|
@ -1,5 +1,5 @@
|
||||
SET max_bytes_before_external_group_by = 100000000;
|
||||
SET max_memory_usage = 201000000;
|
||||
SET max_memory_usage = 351000000;
|
||||
|
||||
SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k);
|
||||
SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k);
|
||||
|
@ -3,17 +3,18 @@
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
9
|
||||
< X-ClickHouse-Progress: {"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"2","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"4","read_bytes":"32","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"5","read_bytes":"40","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"6","read_bytes":"48","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"7","read_bytes":"56","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"8","read_bytes":"64","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"9","read_bytes":"72","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"2","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"4","read_bytes":"32","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"5","read_bytes":"40","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"6","read_bytes":"48","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"7","read_bytes":"56","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"8","read_bytes":"64","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"9","read_bytes":"72","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","written_rows":"0","written_bytes":"0","total_rows_to_read":"10"}
|
||||
0
|
||||
1
|
||||
2
|
||||
|
@ -6,9 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'
|
||||
# This test will fail with external poco (progress not supported)
|
||||
|
||||
# "grep -v 11" in order to skip extra progress header for 11-th row (for processors pipeline)
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&experimental_use_processors=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | grep -v 11
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1&experimental_use_processors=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d
|
||||
|
||||
# 'send_progress_in_http_headers' is false by default
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -q 'X-ClickHouse-Progress' && echo 'Fail' || true
|
||||
|
@ -1,54 +0,0 @@
|
||||
5
|
||||
5
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin
|
||||
13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx
|
||||
25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat
|
||||
3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat
|
||||
37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt
|
||||
38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk
|
||||
55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx
|
||||
5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin
|
||||
f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx
|
||||
5
|
||||
5
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin
|
||||
13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx
|
||||
25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat
|
||||
3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat
|
||||
37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt
|
||||
38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk
|
||||
55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx
|
||||
5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin
|
||||
f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx
|
||||
31,1,2
|
||||
1,2,3
|
@ -1,60 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
# Not found column date in block. There are only columns: x.
|
||||
|
||||
# Test 1. Complex test checking columns.txt
|
||||
|
||||
chl="$CLICKHOUSE_CLIENT -q"
|
||||
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||
|
||||
$chl "DROP TABLE IF EXISTS test.partition_428"
|
||||
$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
# 2 header lines + 3 columns
|
||||
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 FREEZE"
|
||||
|
||||
# Do `cd` for consistent output for reference
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001"
|
||||
$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
# 2 header lines + 3 columns
|
||||
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8"
|
||||
|
||||
# Check the backup hasn't changed
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "OPTIMIZE TABLE test.partition_428"
|
||||
|
||||
$chl "SELECT toUInt16(p), k, v1 FROM test.partition_428 ORDER BY k FORMAT CSV"
|
||||
$chl "DROP TABLE test.partition_428"
|
||||
|
||||
# Test 2. Simple test
|
||||
|
||||
$chl "drop table if exists test.partition_428"
|
||||
$chl "create table test.partition_428 (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) SETTINGS index_granularity=8192, index_granularity_bytes=0"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now() )"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now()+1 )"
|
||||
$chl "alter table test.partition_428 detach partition 197001"
|
||||
$chl "alter table test.partition_428 attach partition 197001"
|
||||
$chl "optimize table test.partition_428"
|
||||
$chl "drop table test.partition_428"
|
@ -1,5 +1,7 @@
|
||||
0
|
||||
1 0
|
||||
3 0
|
||||
2 0
|
||||
1
|
||||
1
|
||||
1
|
||||
finished 42 readonly SELECT 2, count() FROM system.numbers
|
||||
1
|
||||
44
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user