ClickHouse/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp

302 lines
11 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Common/HashTable/HashMap.h>
#include <Common/Exception.h>
#include <Interpreters/AggregationCommon.h>
#include <IO/HashingWriteBuffer.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/typeid_cast.h>
2014-03-13 17:44:00 +00:00
namespace ProfileEvents
{
extern const Event MergeTreeDataWriterBlocks;
extern const Event MergeTreeDataWriterBlocksAlreadySorted;
extern const Event MergeTreeDataWriterRows;
extern const Event MergeTreeDataWriterUncompressedBytes;
extern const Event MergeTreeDataWriterCompressedBytes;
}
2014-03-13 17:44:00 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TOO_MANY_PARTS;
}
namespace
2014-03-13 17:44:00 +00:00
{
void buildScatterSelector(
const ColumnRawPtrs & columns,
2017-08-31 19:56:43 +00:00
PODArray<size_t> & partition_num_to_first_row,
IColumn::Selector & selector,
size_t max_parts)
{
2017-08-31 19:56:43 +00:00
/// Use generic hashed variant since partitioning is unlikely to be a bottleneck.
using Data = HashMap<UInt128, size_t, UInt128TrivialHash>;
Data partitions_map;
size_t num_rows = columns[0]->size();
size_t partitions_count = 0;
for (size_t i = 0; i < num_rows; ++i)
{
Data::key_type key = hash128(i, columns.size(), columns);
typename Data::LookupResult it;
bool inserted;
partitions_map.emplace(key, it, inserted);
if (inserted)
{
if (max_parts && partitions_count >= max_parts)
throw Exception("Too many partitions for single INSERT block (more than " + toString(max_parts) + "). The limit is controlled by 'max_partitions_per_insert_block' setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).", ErrorCodes::TOO_MANY_PARTS);
2017-08-31 19:56:43 +00:00
partition_num_to_first_row.push_back(i);
2019-10-29 15:16:51 +00:00
it->getMapped() = partitions_count;
++partitions_count;
/// Optimization for common case when there is only one partition - defer selector initialization.
if (partitions_count == 2)
{
selector = IColumn::Selector(num_rows);
std::fill(selector.begin(), selector.begin() + i, 0);
}
}
if (partitions_count > 1)
2019-10-29 15:16:51 +00:00
selector[i] = it->getMapped();
}
}
/// Computes ttls and updates ttl infos
void updateTTL(const MergeTreeData::TTLEntry & ttl_entry,
MergeTreeDataPart::TTLInfos & ttl_infos,
DB::MergeTreeDataPartTTLInfo & ttl_info,
Block & block)
{
if (!block.has(ttl_entry.result_column))
ttl_entry.expression->execute(block);
const auto & current = block.getByName(ttl_entry.result_column);
const IColumn * column = current.column.get();
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
{
const auto & date_lut = DateLUT::instance();
for (const auto & val : column_date->getData())
ttl_info.update(date_lut.fromDayNum(DayNum(val)));
}
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
{
for (const auto & val : column_date_time->getData())
ttl_info.update(val);
}
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
{
const auto & date_lut = DateLUT::instance();
ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>())));
}
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
{
ttl_info.update(column_const->getValue<UInt32>());
}
else
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
else
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
}
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts)
{
BlocksWithPartition result;
if (!block || !block.rows())
return result;
data.check(block, true);
block.checkNumberOfRows();
if (!data.partition_key_expr) /// Table is not partitioned.
{
result.emplace_back(Block(block), Row());
return result;
}
Block block_copy = block;
data.partition_key_expr->execute(block_copy);
ColumnRawPtrs partition_columns;
partition_columns.reserve(data.partition_key_sample.columns());
for (const ColumnWithTypeAndName & element : data.partition_key_sample)
partition_columns.emplace_back(block_copy.getByName(element.name).column.get());
2017-08-31 19:56:43 +00:00
PODArray<size_t> partition_num_to_first_row;
IColumn::Selector selector;
buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts);
2017-08-31 19:56:43 +00:00
size_t partitions_count = partition_num_to_first_row.size();
result.reserve(partitions_count);
auto get_partition = [&](size_t num)
{
2017-08-31 19:56:43 +00:00
Row partition(partition_columns.size());
for (size_t i = 0; i < partition_columns.size(); ++i)
2017-08-31 19:56:43 +00:00
partition[i] = Field((*partition_columns[i])[partition_num_to_first_row[num]]);
return partition;
};
if (partitions_count == 1)
{
/// A typical case is when there is one partition (you do not need to split anything).
/// NOTE: returning a copy of the original block so that calculated partition key columns
/// do not interfere with possible calculated primary key columns of the same name.
result.emplace_back(Block(block), get_partition(0));
return result;
}
for (size_t i = 0; i < partitions_count; ++i)
result.emplace_back(block.cloneEmpty(), get_partition(i));
for (size_t col = 0; col < block.columns(); ++col)
{
MutableColumns scattered = block.getByPosition(col).column->scatter(partitions_count, selector);
for (size_t i = 0; i < partitions_count; ++i)
result[i].block.getByPosition(col).column = std::move(scattered[i]);
}
return result;
2014-03-13 17:44:00 +00:00
}
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition)
2014-03-13 17:44:00 +00:00
{
Block & block = block_with_partition.block;
static const String TMP_PREFIX = "tmp_insert_";
/// This will generate unique name in scope of current server process.
Int64 temp_index = data.insert_increment.get();
MergeTreeDataPart::MinMaxIndex minmax_idx;
minmax_idx.update(block, data.minmax_idx_columns);
MergeTreePartition partition(std::move(block_with_partition.partition));
2014-03-13 17:44:00 +00:00
MergeTreePartInfo new_part_info(partition.getID(data.partition_key_sample), temp_index, temp_index, 0);
String part_name;
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
DayNum min_date(minmax_idx.parallelogram[data.minmax_idx_date_column_pos].left.get<UInt64>());
DayNum max_date(minmax_idx.parallelogram[data.minmax_idx_date_column_pos].right.get<UInt64>());
2014-03-13 17:44:00 +00:00
const auto & date_lut = DateLUT::instance();
2015-08-17 21:09:36 +00:00
DayNum min_month = date_lut.toFirstDayNumOfMonth(DayNum(min_date));
DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date));
2015-08-17 21:09:36 +00:00
if (min_month != max_month)
throw Exception("Logical error: part spans more than one month.", ErrorCodes::LOGICAL_ERROR);
2014-03-13 17:44:00 +00:00
part_name = new_part_info.getPartNameV0(min_date, max_date);
}
else
part_name = new_part_info.getPartName();
/// Size of part would not be greater than block.bytes() + epsilon
size_t expected_size = block.bytes();
DB::MergeTreeDataPart::TTLInfos move_ttl_infos;
for (const auto & [expression, ttl_entry] : data.move_ttl_entries_by_name)
{
updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[expression], block);
}
DiskSpace::ReservationPtr reservation = data.reserveSpacePreferringMoveDestination(expected_size, move_ttl_infos, time(nullptr));
2019-09-09 17:01:19 +00:00
MergeTreeData::MutableDataPartPtr new_data_part =
std::make_shared<MergeTreeData::DataPart>(data, reservation->getDisk(), part_name, new_part_info);
new_data_part->partition = std::move(partition);
new_data_part->minmax_idx = std::move(minmax_idx);
new_data_part->relative_path = TMP_PREFIX + part_name;
new_data_part->is_temp = true;
2014-07-17 10:44:17 +00:00
/// The name could be non-unique in case of stale files from previous runs.
String full_path = new_data_part->getFullPath();
Poco::File dir(full_path);
if (dir.exists())
{
LOG_WARNING(log, "Removing old temporary directory " + full_path);
dir.remove(true);
}
dir.createDirectories();
/// If we need to calculate some columns to sort.
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (data.hasSortingKey() || data.hasSkipIndices())
data.sorting_key_and_skip_indices_expr->execute(block);
2014-03-13 17:44:00 +00:00
Names sort_columns = data.sorting_key_columns;
2018-06-30 21:35:01 +00:00
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);
2014-03-14 17:03:52 +00:00
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
2016-04-23 02:39:40 +00:00
/// Sort
IColumn::Permutation * perm_ptr = nullptr;
IColumn::Permutation perm;
if (!sort_description.empty())
{
2018-06-30 21:35:01 +00:00
if (!isAlreadySorted(block, sort_description))
{
2018-06-30 21:35:01 +00:00
stableGetPermutation(block, sort_description, perm);
perm_ptr = &perm;
}
else
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
}
2014-03-13 17:44:00 +00:00
if (data.hasTableTTL())
updateTTL(data.ttl_table_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block);
for (const auto & [name, ttl_entry] : data.ttl_entries_by_name)
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block);
new_data_part->ttl_infos.update(move_ttl_infos);
2017-08-01 20:07:16 +00:00
/// This effectively chooses minimal compression method:
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
auto compression_codec = data.global_context.chooseCompressionCodec(0, 0);
2017-07-31 11:05:49 +00:00
2018-03-13 15:00:28 +00:00
NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames());
2018-12-21 12:17:30 +00:00
MergedBlockOutputStream out(data, new_data_part->getFullPath(), columns, compression_codec);
out.writePrefix();
out.writeWithPermutation(block, perm_ptr);
out.writeSuffixAndFinalizePart(new_data_part);
2014-03-13 17:44:00 +00:00
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows());
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes());
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->bytes_on_disk);
2016-04-23 02:39:40 +00:00
return new_data_part;
2014-03-13 17:44:00 +00:00
}
}