mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'master' into fix_watch_race_testkeeper
This commit is contained in:
commit
dff71850a8
@ -401,6 +401,7 @@ class IColumn;
|
||||
M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \
|
||||
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
|
||||
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
||||
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
|
||||
\
|
||||
M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated parser", 0) \
|
||||
\
|
||||
|
@ -30,7 +30,6 @@ struct SortCursorImpl
|
||||
ColumnRawPtrs all_columns;
|
||||
SortDescription desc;
|
||||
size_t sort_columns_size = 0;
|
||||
size_t pos = 0;
|
||||
size_t rows = 0;
|
||||
|
||||
/** Determines order if comparing columns are equal.
|
||||
@ -49,15 +48,20 @@ struct SortCursorImpl
|
||||
/** Is there at least one column with Collator. */
|
||||
bool has_collation = false;
|
||||
|
||||
/** We could use SortCursorImpl in case when columns aren't sorted
|
||||
* but we have their sorted permutation
|
||||
*/
|
||||
IColumn::Permutation * permutation = nullptr;
|
||||
|
||||
SortCursorImpl() {}
|
||||
|
||||
SortCursorImpl(const Block & block, const SortDescription & desc_, size_t order_ = 0)
|
||||
SortCursorImpl(const Block & block, const SortDescription & desc_, size_t order_ = 0, IColumn::Permutation * perm = nullptr)
|
||||
: desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size())
|
||||
{
|
||||
reset(block);
|
||||
reset(block, perm);
|
||||
}
|
||||
|
||||
SortCursorImpl(const Columns & columns, const SortDescription & desc_, size_t order_ = 0)
|
||||
SortCursorImpl(const Columns & columns, const SortDescription & desc_, size_t order_ = 0, IColumn::Permutation * perm = nullptr)
|
||||
: desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size())
|
||||
{
|
||||
for (auto & column_desc : desc)
|
||||
@ -66,19 +70,19 @@ struct SortCursorImpl
|
||||
throw Exception("SortDescription should contain column position if SortCursor was used without header.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
reset(columns, {});
|
||||
reset(columns, {}, perm);
|
||||
}
|
||||
|
||||
bool empty() const { return rows == 0; }
|
||||
|
||||
/// Set the cursor to the beginning of the new block.
|
||||
void reset(const Block & block)
|
||||
void reset(const Block & block, IColumn::Permutation * perm = nullptr)
|
||||
{
|
||||
reset(block.getColumns(), block);
|
||||
reset(block.getColumns(), block, perm);
|
||||
}
|
||||
|
||||
/// Set the cursor to the beginning of the new block.
|
||||
void reset(const Columns & columns, const Block & block)
|
||||
void reset(const Columns & columns, const Block & block, IColumn::Permutation * perm = nullptr)
|
||||
{
|
||||
all_columns.clear();
|
||||
sort_columns.clear();
|
||||
@ -96,18 +100,33 @@ struct SortCursorImpl
|
||||
: column_desc.column_number;
|
||||
sort_columns.push_back(columns[column_number].get());
|
||||
|
||||
need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->isCollationSupported(); /// TODO Nullable(String)
|
||||
need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->isCollationSupported();
|
||||
has_collation |= need_collation[j];
|
||||
}
|
||||
|
||||
pos = 0;
|
||||
rows = all_columns[0]->size();
|
||||
permutation = perm;
|
||||
}
|
||||
|
||||
size_t getRow() const
|
||||
{
|
||||
if (permutation)
|
||||
return (*permutation)[pos];
|
||||
return pos;
|
||||
}
|
||||
|
||||
/// We need a possibility to change pos (see MergeJoin).
|
||||
size_t & getPosRef() { return pos; }
|
||||
|
||||
bool isFirst() const { return pos == 0; }
|
||||
bool isLast() const { return pos + 1 >= rows; }
|
||||
bool isValid() const { return pos < rows; }
|
||||
void next() { ++pos; }
|
||||
|
||||
/// Prevent using pos instead of getRow()
|
||||
private:
|
||||
size_t pos;
|
||||
};
|
||||
|
||||
using SortCursorImpls = std::vector<SortCursorImpl>;
|
||||
@ -127,7 +146,7 @@ struct SortCursorHelper
|
||||
|
||||
bool ALWAYS_INLINE greater(const SortCursorHelper & rhs) const
|
||||
{
|
||||
return derived().greaterAt(rhs.derived(), impl->pos, rhs.impl->pos);
|
||||
return derived().greaterAt(rhs.derived(), impl->getRow(), rhs.impl->getRow());
|
||||
}
|
||||
|
||||
/// Inverted so that the priority queue elements are removed in ascending order.
|
||||
|
@ -222,7 +222,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort
|
||||
// std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n";
|
||||
// std::cerr << "Inserting row\n";
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
|
||||
merged_columns[i]->insertFrom(*current->all_columns[i], current->getRow());
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
|
@ -1552,7 +1552,12 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// Specify the number of threads only if it wasn't specified in storage.
|
||||
if (!query_plan.getMaxThreads())
|
||||
///
|
||||
/// But in case of remote query and prefer_localhost_replica=1 (default)
|
||||
/// The inner local query (that is done in the same process, without
|
||||
/// network interaction), it will setMaxThreads earlier and distributed
|
||||
/// query will not update it.
|
||||
if (!query_plan.getMaxThreads() || is_remote)
|
||||
query_plan.setMaxThreads(max_threads_execute_query);
|
||||
|
||||
/// Aliases in table declaration.
|
||||
|
@ -180,12 +180,16 @@ class MergeJoinCursor
|
||||
public:
|
||||
MergeJoinCursor(const Block & block, const SortDescription & desc_)
|
||||
: impl(SortCursorImpl(block, desc_))
|
||||
{}
|
||||
{
|
||||
/// SortCursorImpl can work with permutation, but MergeJoinCursor can't.
|
||||
if (impl.permutation)
|
||||
throw Exception("Logical error: MergeJoinCursor doesn't support permutation", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
size_t position() const { return impl.pos; }
|
||||
size_t position() const { return impl.getRow(); }
|
||||
size_t end() const { return impl.rows; }
|
||||
bool atEnd() const { return impl.pos >= impl.rows; }
|
||||
void nextN(size_t num) { impl.pos += num; }
|
||||
bool atEnd() const { return impl.getRow() >= impl.rows; }
|
||||
void nextN(size_t num) { impl.getPosRef() += num; }
|
||||
|
||||
void setCompareNullability(const MergeJoinCursor & rhs)
|
||||
{
|
||||
@ -254,10 +258,10 @@ private:
|
||||
else if (cmp > 0)
|
||||
rhs.impl.next();
|
||||
else if (!cmp)
|
||||
return Range{impl.pos, rhs.impl.pos, getEqualLength(), rhs.getEqualLength()};
|
||||
return Range{impl.getRow(), rhs.impl.getRow(), getEqualLength(), rhs.getEqualLength()};
|
||||
}
|
||||
|
||||
return Range{impl.pos, rhs.impl.pos, 0, 0};
|
||||
return Range{impl.getRow(), rhs.impl.getRow(), 0, 0};
|
||||
}
|
||||
|
||||
template <bool left_nulls, bool right_nulls>
|
||||
@ -268,7 +272,7 @@ private:
|
||||
const auto * left_column = impl.sort_columns[i];
|
||||
const auto * right_column = rhs.impl.sort_columns[i];
|
||||
|
||||
int res = nullableCompareAt<left_nulls, right_nulls>(*left_column, *right_column, impl.pos, rhs.impl.pos);
|
||||
int res = nullableCompareAt<left_nulls, right_nulls>(*left_column, *right_column, impl.getRow(), rhs.impl.getRow());
|
||||
if (res)
|
||||
return res;
|
||||
}
|
||||
@ -278,11 +282,11 @@ private:
|
||||
/// Expects !atEnd()
|
||||
size_t getEqualLength()
|
||||
{
|
||||
size_t pos = impl.pos + 1;
|
||||
size_t pos = impl.getRow() + 1;
|
||||
for (; pos < impl.rows; ++pos)
|
||||
if (!samePrev(pos))
|
||||
break;
|
||||
return pos - impl.pos;
|
||||
return pos - impl.getRow();
|
||||
}
|
||||
|
||||
/// Expects lhs_pos > 0
|
||||
|
@ -257,12 +257,12 @@ void AggregatingSortedAlgorithm::AggregatingMergedData::addRow(SortCursor & curs
|
||||
throw Exception("Can't add a row to the group because it was not started.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos);
|
||||
desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->getRow());
|
||||
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
{
|
||||
auto & col = cursor->all_columns[desc.column_number];
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get());
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->getRow(), arena.get());
|
||||
}
|
||||
}
|
||||
|
||||
@ -352,7 +352,7 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge()
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
merged_data.startGroup(current->all_columns, current->pos);
|
||||
merged_data.startGroup(current->all_columns, current->getRow());
|
||||
}
|
||||
|
||||
merged_data.addRow(current);
|
||||
|
@ -26,9 +26,9 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm(
|
||||
const String & sign_column,
|
||||
bool only_positive_sign_,
|
||||
size_t max_block_size,
|
||||
Poco::Logger * log_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes,
|
||||
Poco::Logger * log_)
|
||||
bool use_average_block_sizes)
|
||||
: IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs)
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, sign_column_number(header.getPositionByName(sign_column))
|
||||
@ -123,7 +123,7 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge()
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->getRow()];
|
||||
|
||||
RowRef current_row;
|
||||
setRowRef(current_row, current);
|
||||
|
@ -33,9 +33,9 @@ public:
|
||||
const String & sign_column,
|
||||
bool only_positive_sign_, /// For select final. Skip rows with sum(sign) < 0.
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes,
|
||||
Poco::Logger * log_);
|
||||
Poco::Logger * log_,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
|
@ -164,12 +164,12 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
|
||||
StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->pos);
|
||||
StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow());
|
||||
bool new_path = is_first || next_path != current_group_path;
|
||||
|
||||
is_first = false;
|
||||
|
||||
time_t next_row_time = current->all_columns[columns_definition.time_column_num]->getUInt(current->pos);
|
||||
time_t next_row_time = current->all_columns[columns_definition.time_column_num]->getUInt(current->getRow());
|
||||
/// Is new key before rounding.
|
||||
bool is_new_key = new_path || next_row_time != current_time;
|
||||
|
||||
@ -227,7 +227,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
/// and for rows with same maximum version - only last row.
|
||||
if (is_new_key
|
||||
|| current->all_columns[columns_definition.version_column_num]->compareAt(
|
||||
current->pos, current_subgroup_newest_row.row_num,
|
||||
current->getRow(), current_subgroup_newest_row.row_num,
|
||||
*(*current_subgroup_newest_row.all_columns)[columns_definition.version_column_num],
|
||||
/* nan_direction_hint = */ 1) >= 0)
|
||||
{
|
||||
@ -263,7 +263,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
|
||||
void GraphiteRollupSortedAlgorithm::startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule)
|
||||
{
|
||||
merged_data.startNextGroup(cursor->all_columns, cursor->pos, next_rule, columns_definition);
|
||||
merged_data.startNextGroup(cursor->all_columns, cursor->getRow(), next_rule, columns_definition);
|
||||
}
|
||||
|
||||
void GraphiteRollupSortedAlgorithm::finishCurrentGroup()
|
||||
|
@ -29,6 +29,8 @@ public:
|
||||
/// between different algorithm objects in parallel FINAL.
|
||||
bool skip_last_row = false;
|
||||
|
||||
IColumn::Permutation * permutation = nullptr;
|
||||
|
||||
void swap(Input & other)
|
||||
{
|
||||
chunk.swap(other.chunk);
|
||||
|
@ -22,7 +22,7 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Inputs inputs)
|
||||
if (!current_inputs[source_num].chunk)
|
||||
continue;
|
||||
|
||||
cursors[source_num] = SortCursorImpl(current_inputs[source_num].chunk.getColumns(), description, source_num);
|
||||
cursors[source_num] = SortCursorImpl(current_inputs[source_num].chunk.getColumns(), description, source_num, current_inputs[source_num].permutation);
|
||||
}
|
||||
|
||||
queue = SortingHeap<SortCursor>(cursors);
|
||||
@ -37,7 +37,7 @@ void IMergingAlgorithmWithDelayedChunk::updateCursor(Input & input, size_t sourc
|
||||
last_chunk_sort_columns = std::move(cursors[source_num].sort_columns);
|
||||
|
||||
current_input.swap(input);
|
||||
cursors[source_num].reset(current_input.chunk.getColumns(), {});
|
||||
cursors[source_num].reset(current_input.chunk.getColumns(), {}, current_input.permutation);
|
||||
|
||||
queue.push(cursors[source_num]);
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ void IMergingAlgorithmWithSharedChunks::initialize(Inputs inputs)
|
||||
|
||||
source.skip_last_row = inputs[source_num].skip_last_row;
|
||||
source.chunk = chunk_allocator.alloc(inputs[source_num].chunk);
|
||||
cursors[source_num] = SortCursorImpl(source.chunk->getColumns(), description, source_num);
|
||||
cursors[source_num] = SortCursorImpl(source.chunk->getColumns(), description, source_num, inputs[source_num].permutation);
|
||||
|
||||
source.chunk->all_columns = cursors[source_num].all_columns;
|
||||
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
@ -55,7 +55,7 @@ void IMergingAlgorithmWithSharedChunks::consume(Input & input, size_t source_num
|
||||
auto & source = sources[source_num];
|
||||
source.skip_last_row = input.skip_last_row;
|
||||
source.chunk = chunk_allocator.alloc(input.chunk);
|
||||
cursors[source_num].reset(source.chunk->getColumns(), {});
|
||||
cursors[source_num].reset(source.chunk->getColumns(), {}, input.permutation);
|
||||
|
||||
source.chunk->all_columns = cursors[source_num].all_columns;
|
||||
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
|
@ -139,7 +139,7 @@ IMergingAlgorithm::Status MergingSortedAlgorithm::mergeImpl(TSortingHeap & queue
|
||||
|
||||
//std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n";
|
||||
//std::cerr << "Inserting row\n";
|
||||
merged_data.insertRow(current->all_columns, current->pos, current->rows);
|
||||
merged_data.insertRow(current->all_columns, current->getRow(), current->rows);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
|
@ -18,9 +18,9 @@ public:
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes);
|
||||
UInt64 limit_ = 0,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
|
||||
void addInput();
|
||||
|
||||
|
@ -73,7 +73,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge()
|
||||
if (version_column_number == -1
|
||||
|| selected_row.empty()
|
||||
|| current->all_columns[version_column_number]->compareAt(
|
||||
current->pos, selected_row.row_num,
|
||||
current->getRow(), selected_row.row_num,
|
||||
*(*selected_row.all_columns)[version_column_number],
|
||||
/* nan_direction_hint = */ 1) >= 0)
|
||||
{
|
||||
|
@ -136,7 +136,7 @@ struct RowRef
|
||||
{
|
||||
sort_columns = cursor.impl->sort_columns.data();
|
||||
num_columns = cursor.impl->sort_columns.size();
|
||||
row_num = cursor.impl->pos;
|
||||
row_num = cursor.impl->getRow();
|
||||
}
|
||||
|
||||
static bool checkEquals(size_t size, const IColumn ** lhs, size_t lhs_row, const IColumn ** rhs, size_t rhs_row)
|
||||
@ -192,7 +192,7 @@ struct RowRefWithOwnedChunk
|
||||
void set(SortCursor & cursor, SharedChunkPtr chunk)
|
||||
{
|
||||
owned_chunk = std::move(chunk);
|
||||
row_num = cursor.impl->pos;
|
||||
row_num = cursor.impl->getRow();
|
||||
all_columns = &owned_chunk->all_columns;
|
||||
sort_columns = &owned_chunk->sort_columns;
|
||||
}
|
||||
|
@ -688,10 +688,10 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge()
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
merged_data.startGroup(current->all_columns, current->pos);
|
||||
merged_data.startGroup(current->all_columns, current->getRow());
|
||||
}
|
||||
else
|
||||
merged_data.addRow(current->all_columns, current->pos);
|
||||
merged_data.addRow(current->all_columns, current->getRow());
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
|
@ -73,7 +73,7 @@ IMergingAlgorithm::Status VersionedCollapsingAlgorithm::merge()
|
||||
|
||||
RowRef current_row;
|
||||
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->getRow()];
|
||||
|
||||
setRowRef(current_row, current);
|
||||
|
||||
|
@ -27,9 +27,9 @@ public:
|
||||
sign_column,
|
||||
only_positive_sign,
|
||||
max_block_size,
|
||||
&Poco::Logger::get("CollapsingSortedTransform"),
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes,
|
||||
&Poco::Logger::get("CollapsingSortedTransform"))
|
||||
use_average_block_sizes)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -100,7 +100,7 @@ Chunk MergeSorter::mergeImpl(TSortingHeap & queue)
|
||||
|
||||
/// Append a row from queue.
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
|
||||
merged_columns[i]->insertFrom(*current->all_columns[i], current->getRow());
|
||||
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
|
@ -22,7 +22,7 @@ void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot);
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, optimize_on_insert);
|
||||
storage.renameTempPartAndAdd(part, &storage.increment);
|
||||
|
||||
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
|
||||
|
@ -14,10 +14,11 @@ class StorageMergeTree;
|
||||
class MergeTreeBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_)
|
||||
MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_, bool optimize_on_insert_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, max_parts_per_block(max_parts_per_block_)
|
||||
, optimize_on_insert(optimize_on_insert_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -28,6 +29,7 @@ private:
|
||||
StorageMergeTree & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
size_t max_parts_per_block;
|
||||
bool optimize_on_insert;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -16,6 +16,14 @@
|
||||
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/MergingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/SummingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MergeTreeDataWriterBlocks;
|
||||
@ -194,7 +202,74 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
return result;
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot)
|
||||
Block MergeTreeDataWriter::mergeBlock(const Block & block, SortDescription sort_description, Names & partition_key_columns, IColumn::Permutation *& permutation)
|
||||
{
|
||||
size_t block_size = block.rows();
|
||||
|
||||
auto get_merging_algorithm = [&]() -> std::shared_ptr<IMergingAlgorithm>
|
||||
{
|
||||
switch (data.merging_params.mode)
|
||||
{
|
||||
/// There is nothing to merge in single block in ordinary MergeTree
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
return nullptr;
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
return std::make_shared<ReplacingSortedAlgorithm>(
|
||||
block, 1, sort_description, data.merging_params.version_column, block_size + 1);
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
return std::make_shared<CollapsingSortedAlgorithm>(
|
||||
block, 1, sort_description, data.merging_params.sign_column,
|
||||
false, block_size + 1, &Poco::Logger::get("MergeTreeBlockOutputStream"));
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
return std::make_shared<SummingSortedAlgorithm>(
|
||||
block, 1, sort_description, data.merging_params.columns_to_sum,
|
||||
partition_key_columns, block_size + 1);
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
return std::make_shared<AggregatingSortedAlgorithm>(block, 1, sort_description, block_size + 1);
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
return std::make_shared<VersionedCollapsingAlgorithm>(
|
||||
block, 1, sort_description, data.merging_params.sign_column, block_size + 1);
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
return std::make_shared<GraphiteRollupSortedAlgorithm>(
|
||||
block, 1, sort_description, block_size + 1, data.merging_params.graphite_params, time(nullptr));
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
};
|
||||
|
||||
auto merging_algorithm = get_merging_algorithm();
|
||||
if (!merging_algorithm)
|
||||
return block;
|
||||
|
||||
Chunk chunk(block.getColumns(), block_size);
|
||||
|
||||
IMergingAlgorithm::Input input;
|
||||
input.set(std::move(chunk));
|
||||
input.permutation = permutation;
|
||||
|
||||
IMergingAlgorithm::Inputs inputs;
|
||||
inputs.push_back(std::move(input));
|
||||
merging_algorithm->initialize(std::move(inputs));
|
||||
|
||||
IMergingAlgorithm::Status status = merging_algorithm->merge();
|
||||
|
||||
/// Check that after first merge merging_algorithm is waiting for data from input 0.
|
||||
if (status.required_source != 0)
|
||||
throw Exception("Logical error: required source after the first merge is not 0.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
status = merging_algorithm->merge();
|
||||
|
||||
/// Check that merge is finished.
|
||||
if (!status.is_finished)
|
||||
throw Exception("Logical error: merge is not finished after the second merge.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// Merged Block is sorted and we don't need to use permutation anymore
|
||||
permutation = nullptr;
|
||||
|
||||
return block.cloneWithColumns(status.chunk.getColumns());
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, bool optimize_on_insert)
|
||||
{
|
||||
Block & block = block_with_partition.block;
|
||||
|
||||
@ -228,6 +303,38 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
else
|
||||
part_name = new_part_info.getPartName();
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices())
|
||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block);
|
||||
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||
sort_description.emplace_back(block.getPositionByName(sort_columns[i]), 1, 1);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
|
||||
|
||||
/// Sort
|
||||
IColumn::Permutation * perm_ptr = nullptr;
|
||||
IColumn::Permutation perm;
|
||||
if (!sort_description.empty())
|
||||
{
|
||||
if (!isAlreadySorted(block, sort_description))
|
||||
{
|
||||
stableGetPermutation(block, sort_description, perm);
|
||||
perm_ptr = &perm;
|
||||
}
|
||||
else
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names;
|
||||
if (optimize_on_insert)
|
||||
block = mergeBlock(block, sort_description, partition_key_columns, perm_ptr);
|
||||
|
||||
/// Size of part would not be greater than block.bytes() + epsilon
|
||||
size_t expected_size = block.bytes();
|
||||
|
||||
@ -274,34 +381,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
sync_guard.emplace(disk, full_path);
|
||||
}
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices())
|
||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block);
|
||||
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||
sort_description.emplace_back(block.getPositionByName(sort_columns[i]), 1, 1);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
|
||||
|
||||
/// Sort
|
||||
IColumn::Permutation * perm_ptr = nullptr;
|
||||
IColumn::Permutation perm;
|
||||
if (!sort_description.empty())
|
||||
{
|
||||
if (!isAlreadySorted(block, sort_description))
|
||||
{
|
||||
stableGetPermutation(block, sort_description, perm);
|
||||
perm_ptr = &perm;
|
||||
}
|
||||
else
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
if (metadata_snapshot->hasRowsTTL())
|
||||
updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
|
||||
|
||||
|
@ -45,7 +45,9 @@ public:
|
||||
/** All rows must correspond to same partition.
|
||||
* Returns part with unique name starting with 'tmp_', yet not added to MergeTreeData.
|
||||
*/
|
||||
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot);
|
||||
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, bool optimize_on_insert);
|
||||
|
||||
Block mergeBlock(const Block & block, SortDescription sort_description, Names & partition_key_columns, IColumn::Permutation *& permutation);
|
||||
|
||||
private:
|
||||
MergeTreeData & data;
|
||||
|
@ -40,7 +40,8 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream(
|
||||
size_t quorum_timeout_ms_,
|
||||
size_t max_parts_per_block_,
|
||||
bool quorum_parallel_,
|
||||
bool deduplicate_)
|
||||
bool deduplicate_,
|
||||
bool optimize_on_insert_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, quorum(quorum_)
|
||||
@ -49,6 +50,7 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream(
|
||||
, quorum_parallel(quorum_parallel_)
|
||||
, deduplicate(deduplicate_)
|
||||
, log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)"))
|
||||
, optimize_on_insert(optimize_on_insert_)
|
||||
{
|
||||
/// The quorum value `1` has the same meaning as if it is disabled.
|
||||
if (quorum == 1)
|
||||
@ -142,7 +144,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
|
||||
|
||||
/// Write part to the filesystem under temporary name. Calculate a checksum.
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot);
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, optimize_on_insert);
|
||||
|
||||
String block_id;
|
||||
|
||||
|
@ -29,7 +29,8 @@ public:
|
||||
size_t quorum_timeout_ms_,
|
||||
size_t max_parts_per_block_,
|
||||
bool quorum_parallel_,
|
||||
bool deduplicate_);
|
||||
bool deduplicate_,
|
||||
bool optimize_on_insert);
|
||||
|
||||
Block getHeader() const override;
|
||||
void writePrefix() override;
|
||||
@ -71,6 +72,8 @@ private:
|
||||
|
||||
using Logger = Poco::Logger;
|
||||
Poco::Logger * log;
|
||||
|
||||
bool optimize_on_insert;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -84,22 +84,6 @@ StorageMerge::StorageMerge(
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
StorageMerge::StorageMerge(
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const Tables & tables_,
|
||||
const Context & context_)
|
||||
: IStorage(table_id_)
|
||||
, source_database(source_database_)
|
||||
, tables(tables_)
|
||||
, global_context(context_.getGlobalContext())
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
StoragePtr StorageMerge::getFirstTable(F && predicate) const
|
||||
{
|
||||
@ -455,12 +439,8 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont
|
||||
e.addMessage("while getting table iterator of Merge table. Maybe caused by two Merge tables that will endlessly try to read each other's data");
|
||||
throw;
|
||||
}
|
||||
|
||||
if (tables)
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(*tables, source_database);
|
||||
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp->match(table_name_); };
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); };
|
||||
return database->getTablesIterator(context, table_name_match);
|
||||
}
|
||||
|
||||
|
@ -48,8 +48,7 @@ public:
|
||||
|
||||
private:
|
||||
String source_database;
|
||||
std::optional<OptimizedRegularExpression> table_name_regexp;
|
||||
std::optional<Tables> tables;
|
||||
OptimizedRegularExpression table_name_regexp;
|
||||
const Context & global_context;
|
||||
|
||||
using StorageWithLockAndName = std::tuple<StoragePtr, TableLockHolder, String>;
|
||||
@ -76,13 +75,6 @@ protected:
|
||||
const String & table_name_regexp_,
|
||||
const Context & context_);
|
||||
|
||||
StorageMerge(
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const Tables & source_tables_,
|
||||
const Context & context_);
|
||||
|
||||
Pipe createSources(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
|
@ -233,7 +233,7 @@ BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Sto
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
return std::make_shared<MergeTreeBlockOutputStream>(
|
||||
*this, metadata_snapshot, settings.max_partitions_per_insert_block);
|
||||
*this, metadata_snapshot, settings.max_partitions_per_insert_block, context.getSettingsRef().optimize_on_insert);
|
||||
}
|
||||
|
||||
void StorageMergeTree::checkTableCanBeDropped() const
|
||||
|
@ -3861,7 +3861,8 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/,
|
||||
query_settings.insert_quorum_timeout.totalMilliseconds(),
|
||||
query_settings.max_partitions_per_insert_block,
|
||||
query_settings.insert_quorum_parallel,
|
||||
deduplicate);
|
||||
deduplicate,
|
||||
context.getSettingsRef().optimize_on_insert);
|
||||
}
|
||||
|
||||
|
||||
@ -4444,7 +4445,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition(
|
||||
PartsTemporaryRename renamed_parts(*this, "detached/");
|
||||
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts);
|
||||
|
||||
ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false); /// TODO Allow to use quorum here.
|
||||
ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); /// TODO Allow to use quorum here.
|
||||
for (size_t i = 0; i < loaded_parts.size(); ++i)
|
||||
{
|
||||
String old_name = loaded_parts[i]->name;
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
@ -23,6 +22,29 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context)
|
||||
{
|
||||
OptimizedRegularExpression table_name_regexp(table_name_regexp_);
|
||||
auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); };
|
||||
|
||||
StoragePtr any_table;
|
||||
|
||||
{
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
auto iterator = database->getTablesIterator(context, table_name_match);
|
||||
|
||||
if (iterator->isValid())
|
||||
if (const auto & table = iterator->table())
|
||||
any_table = table;
|
||||
}
|
||||
|
||||
if (!any_table)
|
||||
throw Exception("Error while executing table function merge. In database " + source_database + " no one matches regular expression: "
|
||||
+ table_name_regexp_, ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical();
|
||||
}
|
||||
|
||||
void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context)
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
@ -46,46 +68,9 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Conte
|
||||
table_name_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
|
||||
|
||||
const Tables & TableFunctionMerge::getMatchingTables(const Context & context) const
|
||||
{
|
||||
if (tables)
|
||||
return *tables;
|
||||
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
|
||||
OptimizedRegularExpression re(table_name_regexp);
|
||||
auto table_name_match = [&](const String & table_name_) { return re.match(table_name_); };
|
||||
|
||||
auto access = context.getAccess();
|
||||
bool granted_show_on_all_tables = access->isGranted(AccessType::SHOW_TABLES, source_database);
|
||||
bool granted_select_on_all_tables = access->isGranted(AccessType::SELECT, source_database);
|
||||
|
||||
tables.emplace();
|
||||
for (auto it = database->getTablesIterator(context, table_name_match); it->isValid(); it->next())
|
||||
{
|
||||
if (!it->table())
|
||||
continue;
|
||||
bool granted_show = granted_show_on_all_tables || access->isGranted(AccessType::SHOW_TABLES, source_database, it->name());
|
||||
if (!granted_show)
|
||||
continue;
|
||||
if (!granted_select_on_all_tables)
|
||||
access->checkAccess(AccessType::SELECT, source_database, it->name());
|
||||
tables->emplace(it->name(), it->table());
|
||||
}
|
||||
|
||||
if (tables->empty())
|
||||
throw Exception("Error while executing table function merge. In database " + source_database + " no one matches regular expression: "
|
||||
+ table_name_regexp, ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return *tables;
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & context) const
|
||||
{
|
||||
auto first_table = getMatchingTables(context).begin()->second;
|
||||
return ColumnsDescription{first_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical()};
|
||||
return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)};
|
||||
}
|
||||
|
||||
StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||
@ -94,7 +79,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, cons
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
getActualTableStructure(context),
|
||||
source_database,
|
||||
getMatchingTables(context),
|
||||
table_name_regexp,
|
||||
context);
|
||||
|
||||
res->startup();
|
||||
|
@ -19,13 +19,11 @@ private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||
const char * getStorageTypeName() const override { return "Merge"; }
|
||||
|
||||
const Tables & getMatchingTables(const Context & context) const;
|
||||
ColumnsDescription getActualTableStructure(const Context & context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, const Context & context) override;
|
||||
|
||||
String source_database;
|
||||
String table_name_regexp;
|
||||
mutable std::optional<Tables> tables;
|
||||
};
|
||||
|
||||
|
||||
|
@ -0,0 +1,8 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<optimize_on_insert>0</optimize_on_insert>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
@ -8,7 +8,8 @@ from helpers.test_tools import TSV
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance',
|
||||
main_configs=['configs/graphite_rollup.xml'])
|
||||
main_configs=['configs/graphite_rollup.xml'],
|
||||
user_configs=["configs/users.xml"])
|
||||
q = instance.query
|
||||
|
||||
|
||||
|
@ -3,6 +3,8 @@
|
||||
<profiles>
|
||||
<default>
|
||||
<allow_experimental_database_materialize_mysql>1</allow_experimental_database_materialize_mysql>
|
||||
<allow_introspection_functions>1</allow_introspection_functions>
|
||||
<optimize_on_insert>0</optimize_on_insert>
|
||||
<default_database_engine>Ordinary</default_database_engine>
|
||||
</default>
|
||||
</profiles>
|
||||
|
@ -1,5 +1,10 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<optimize_on_insert>0</optimize_on_insert>
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<another>
|
||||
<password/>
|
||||
@ -10,4 +15,4 @@
|
||||
<quota>default</quota>
|
||||
</another>
|
||||
</users>
|
||||
</yandex>
|
||||
</yandex>
|
||||
|
@ -1,55 +0,0 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance')
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
instance.query("CREATE TABLE table1(x UInt32) ENGINE = MergeTree ORDER BY tuple()")
|
||||
instance.query("CREATE TABLE table2(x UInt32) ENGINE = MergeTree ORDER BY tuple()")
|
||||
instance.query("INSERT INTO table1 VALUES (1)")
|
||||
instance.query("INSERT INTO table2 VALUES (2)")
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def cleanup_after_test():
|
||||
try:
|
||||
yield
|
||||
finally:
|
||||
instance.query("DROP USER IF EXISTS A")
|
||||
|
||||
|
||||
def test_merge():
|
||||
select_query = "SELECT * FROM merge('default', 'table[0-9]+') ORDER BY x"
|
||||
assert instance.query(select_query) == "1\n2\n"
|
||||
|
||||
instance.query("CREATE USER A")
|
||||
assert "it's necessary to have the grant CREATE TEMPORARY TABLE ON *.*" in instance.query_and_get_error(select_query, user = 'A')
|
||||
|
||||
instance.query("GRANT CREATE TEMPORARY TABLE ON *.* TO A")
|
||||
assert "no one matches regular expression" in instance.query_and_get_error(select_query, user = 'A')
|
||||
|
||||
instance.query("GRANT SELECT ON default.table1 TO A")
|
||||
assert instance.query(select_query, user = 'A') == "1\n"
|
||||
|
||||
instance.query("GRANT SELECT ON default.* TO A")
|
||||
assert instance.query(select_query, user = 'A') == "1\n2\n"
|
||||
|
||||
instance.query("REVOKE SELECT ON default.table1 FROM A")
|
||||
assert instance.query(select_query, user = 'A') == "2\n"
|
||||
|
||||
instance.query("REVOKE ALL ON default.* FROM A")
|
||||
instance.query("GRANT SELECT ON default.table1 TO A")
|
||||
instance.query("GRANT INSERT ON default.table2 TO A")
|
||||
assert "it's necessary to have the grant SELECT ON default.table2" in instance.query_and_get_error(select_query, user = 'A')
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS merge_tree;
|
||||
DROP TABLE IF EXISTS collapsing_merge_tree;
|
||||
DROP TABLE IF EXISTS versioned_collapsing_merge_tree;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS summing_composite_key;
|
||||
CREATE TABLE summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32, k2ID Int8, s Float64), SecondMap Nested(k1ID UInt64, k2Key String, k3Type Int32, s Int64)) ENGINE = SummingMergeTree(d, k, 1);
|
||||
|
||||
|
@ -42,13 +42,13 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT
|
||||
toDate(0) AS date,
|
||||
toInt8(1) AS Sign,
|
||||
toUInt64(0) AS ki
|
||||
FROM system.numbers LIMIT 9000"
|
||||
FROM system.numbers LIMIT 9000" --server_logs_file=/dev/null
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO $name (date, Sign, ki) SELECT
|
||||
toDate(0) AS date,
|
||||
toInt8(1) AS Sign,
|
||||
number AS ki
|
||||
FROM system.numbers LIMIT 9000, 9000"
|
||||
FROM system.numbers LIMIT 9000, 9000" --server_logs_file=/dev/null
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO $name SELECT
|
||||
toDate(0) AS date,
|
||||
@ -67,7 +67,7 @@ number AS di09,
|
||||
number AS di10,
|
||||
[number, number+1] AS \`n.i\`,
|
||||
[hex(number), hex(number+1)] AS \`n.s\`
|
||||
FROM system.numbers LIMIT $res_rows"
|
||||
FROM system.numbers LIMIT $res_rows" --server_logs_file=/dev/null
|
||||
|
||||
while [[ $(get_num_parts) -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
SELECT '*** Replicated with sampling ***';
|
||||
|
||||
DROP TABLE IF EXISTS replicated_with_sampling;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set optimize_on_insert = 0;
|
||||
|
||||
drop table if exists mult_tab;
|
||||
create table mult_tab (date Date, value String, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(date, (date), 8192, sign, version);
|
||||
insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set optimize_on_insert = 0;
|
||||
|
||||
drop table if exists tab_00577;
|
||||
create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0;
|
||||
insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS test_00616;
|
||||
DROP TABLE IF EXISTS replacing_00616;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS partitioned_by_tuple;
|
||||
|
||||
CREATE TABLE partitioned_by_tuple (d Date, x UInt8, w String, y UInt8) ENGINE SummingMergeTree (y) PARTITION BY (d, x) ORDER BY (d, x, w);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661;
|
||||
DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661;
|
||||
CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w);
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS old_style;
|
||||
CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS old_style;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
select '-- SummingMergeTree with Nullable column without duplicates.';
|
||||
|
||||
drop table if exists tst;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS data_01285;
|
||||
|
||||
SET max_threads=1;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS tags;
|
||||
|
||||
CREATE TABLE tags (
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS tt_01373;
|
||||
|
||||
CREATE TABLE tt_01373
|
||||
|
13
tests/queries/0_stateless/01560_optimize_on_insert.reference
Normal file
13
tests/queries/0_stateless/01560_optimize_on_insert.reference
Normal file
@ -0,0 +1,13 @@
|
||||
Replacing Merge Tree
|
||||
1 2020-01-01 00:00:00
|
||||
2 2020-01-02 00:00:00
|
||||
Collapsing Merge Tree
|
||||
1 1 2020-01-01 00:00:00
|
||||
Versioned Collapsing Merge Tree
|
||||
1 1 2 2020-01-01 00:00:00
|
||||
Summing Merge Tree
|
||||
1 6 2020-01-01 00:00:00
|
||||
2 6 2020-01-02 00:00:00
|
||||
Aggregating Merge Tree
|
||||
1 5 2020-01-01 00:00:00
|
||||
2 5 2020-01-02 00:00:00
|
35
tests/queries/0_stateless/01560_optimize_on_insert.sql
Normal file
35
tests/queries/0_stateless/01560_optimize_on_insert.sql
Normal file
@ -0,0 +1,35 @@
|
||||
SELECT 'Replacing Merge Tree';
|
||||
DROP TABLE IF EXISTS replacing_merge_tree;
|
||||
CREATE TABLE replacing_merge_tree (key UInt32, date Datetime) ENGINE=ReplacingMergeTree() PARTITION BY date ORDER BY key;
|
||||
INSERT INTO replacing_merge_tree VALUES (1, '2020-01-01'), (2, '2020-01-02'), (1, '2020-01-01'), (2, '2020-01-02');
|
||||
SELECT * FROM replacing_merge_tree ORDER BY key;
|
||||
DROP TABLE replacing_merge_tree;
|
||||
|
||||
SELECT 'Collapsing Merge Tree';
|
||||
DROP TABLE IF EXISTS collapsing_merge_tree;
|
||||
CREATE TABLE collapsing_merge_tree (key UInt32, sign Int8, date Datetime) ENGINE=CollapsingMergeTree(sign) PARTITION BY date ORDER BY key;
|
||||
INSERT INTO collapsing_merge_tree VALUES (1, 1, '2020-01-01'), (2, 1, '2020-01-02'), (1, -1, '2020-01-01'), (2, -1, '2020-01-02'), (1, 1, '2020-01-01');
|
||||
SELECT * FROM collapsing_merge_tree ORDER BY key;
|
||||
DROP TABLE collapsing_merge_tree;
|
||||
|
||||
SELECT 'Versioned Collapsing Merge Tree';
|
||||
DROP TABLE IF EXISTS versioned_collapsing_merge_tree;
|
||||
CREATE TABLE versioned_collapsing_merge_tree (key UInt32, sign Int8, version Int32, date Datetime) ENGINE=VersionedCollapsingMergeTree(sign, version) PARTITION BY date ORDER BY (key, version);
|
||||
INSERT INTO versioned_collapsing_merge_tree VALUES (1, 1, 1, '2020-01-01'), (1, -1, 1, '2020-01-01'), (1, 1, 2, '2020-01-01');
|
||||
SELECT * FROM versioned_collapsing_merge_tree ORDER BY key;
|
||||
DROP TABLE versioned_collapsing_merge_tree;
|
||||
|
||||
SELECT 'Summing Merge Tree';
|
||||
DROP TABLE IF EXISTS summing_merge_tree;
|
||||
CREATE TABLE summing_merge_tree (key UInt32, val UInt32, date Datetime) ENGINE=SummingMergeTree(val) PARTITION BY date ORDER BY key;
|
||||
INSERT INTO summing_merge_tree VALUES (1, 1, '2020-01-01'), (2, 1, '2020-01-02'), (1, 5, '2020-01-01'), (2, 5, '2020-01-02');
|
||||
SELECT * FROM summing_merge_tree ORDER BY key;
|
||||
DROP TABLE summing_merge_tree;
|
||||
|
||||
SELECT 'Aggregating Merge Tree';
|
||||
DROP TABLE IF EXISTS aggregating_merge_tree;
|
||||
CREATE TABLE aggregating_merge_tree (key UInt32, val SimpleAggregateFunction(max, UInt32), date Datetime) ENGINE=AggregatingMergeTree() PARTITION BY date ORDER BY key;
|
||||
INSERT INTO aggregating_merge_tree VALUES (1, 1, '2020-01-01'), (2, 1, '2020-01-02'), (1, 5, '2020-01-01'), (2, 5, '2020-01-02');
|
||||
SELECT * FROM aggregating_merge_tree ORDER BY key;
|
||||
DROP TABLE aggregating_merge_tree;
|
||||
|
15
tests/queries/0_stateless/01602_max_distributed_connections.sh
Executable file
15
tests/queries/0_stateless/01602_max_distributed_connections.sh
Executable file
@ -0,0 +1,15 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
common_opts=(
|
||||
"--format=Null"
|
||||
|
||||
"--max_threads=1"
|
||||
"--max_distributed_connections=3"
|
||||
)
|
||||
|
||||
# NOTE: the test use higher timeout to avoid flakiness.
|
||||
timeout 9s ${CLICKHOUSE_CLIENT} "$@" "${common_opts[@]}" -q "select sleep(3) from remote('127.{1,2,3,4,5}', system.one)" --prefer_localhost_replica=0
|
||||
timeout 9s ${CLICKHOUSE_CLIENT} "$@" "${common_opts[@]}" -q "select sleep(3) from remote('127.{1,2,3,4,5}', system.one)" --prefer_localhost_replica=1
|
Loading…
Reference in New Issue
Block a user