Merge pull request #10697 from CurtizJ/polymorphic-parts

Polymorphic parts (in-memory format)
This commit is contained in:
alesapin 2020-07-08 13:25:45 +03:00 committed by GitHub
commit 84f8bf1e14
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
96 changed files with 2375 additions and 677 deletions

View File

@ -355,6 +355,13 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const
}
}
void ColumnAggregateFunction::updateHashFast(SipHash & hash) const
{
/// Fallback to per-element hashing, as there is no faster way
for (size_t i = 0; i < size(); ++i)
updateHashWithValue(i, hash);
}
/// The returned size is less than real size. The reason is that some parts of
/// aggregate function data may be allocated on shared arenas. These arenas are
/// used for several blocks, and also may be updated concurrently from other

View File

@ -159,6 +159,8 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
size_t byteSize() const override;
size_t allocatedBytes() const override;

View File

@ -259,6 +259,12 @@ void ColumnArray::updateWeakHash32(WeakHash32 & hash) const
}
}
void ColumnArray::updateHashFast(SipHash & hash) const
{
offsets->updateHashFast(hash);
data->updateHashFast(hash);
}
void ColumnArray::insert(const Field & x)
{
const Array & array = DB::get<const Array &>(x);

View File

@ -63,6 +63,7 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override;
void insertFrom(const IColumn & src_, size_t n) override;

View File

@ -170,6 +170,11 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override
{
data->updateHashFast(hash);
}
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr replicate(const Offsets & offsets) const override;
ColumnPtr permute(const Permutation & perm, size_t limit) const override;

View File

@ -98,6 +98,12 @@ void ColumnDecimal<T>::updateWeakHash32(WeakHash32 & hash) const
}
}
template <typename T>
void ColumnDecimal<T>::updateHashFast(SipHash & hash) const
{
hash.update(reinterpret_cast<const char *>(data.data()), size() * sizeof(data[0]));
}
template <typename T>
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
{

View File

@ -107,6 +107,7 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
void compareColumn(const IColumn & rhs, size_t rhs_row_num,
PaddedPODArray<UInt64> * row_indexes, PaddedPODArray<Int8> & compare_results,

View File

@ -124,6 +124,12 @@ void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const
}
}
void ColumnFixedString::updateHashFast(SipHash & hash) const
{
hash.update(n);
hash.update(reinterpret_cast<const char *>(chars.data()), size() * n);
}
template <bool positive>
struct ColumnFixedString::less
{

View File

@ -111,6 +111,8 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnFixedString & rhs = assert_cast<const ColumnFixedString &>(rhs_);

View File

@ -107,6 +107,11 @@ public:
throw Exception("updateWeakHash32 is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void updateHashFast(SipHash &) const override
{
throw Exception("updateHashFast is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void popBack(size_t) override
{
throw Exception("popBack is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -257,6 +257,12 @@ void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const
idx.updateWeakHash(hash, dict_hash);
}
void ColumnLowCardinality::updateHashFast(SipHash & hash) const
{
idx.getPositions()->updateHashFast(hash);
getDictionary().getNestedColumn()->updateHashFast(hash);
}
void ColumnLowCardinality::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);

View File

@ -93,6 +93,8 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash &) const override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
{
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint));

View File

@ -64,6 +64,12 @@ void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const
hash_data[row] = old_hash_data[row];
}
void ColumnNullable::updateHashFast(SipHash & hash) const
{
null_map->updateHashFast(hash);
nested_column->updateHashFast(hash);
}
MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
{
MutableColumnPtr new_nested_col = getNestedColumn().cloneResized(new_size);

View File

@ -90,6 +90,7 @@ public:
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
void getExtremes(Field & min, Field & max) const override;
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override

View File

@ -191,6 +191,12 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override
{
hash.update(reinterpret_cast<const char *>(offsets.data()), size() * sizeof(offsets[0]));
hash.update(reinterpret_cast<const char *>(chars.data()), size() * sizeof(chars[0]));
}
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;

View File

@ -195,6 +195,12 @@ void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const
column->updateWeakHash32(hash);
}
void ColumnTuple::updateHashFast(SipHash & hash) const
{
for (const auto & column : columns)
column->updateHashFast(hash);
}
void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const size_t tuple_size = columns.size();

View File

@ -63,6 +63,7 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr permute(const Permutation & perm, size_t limit) const override;

View File

@ -86,6 +86,12 @@ void ColumnVector<T>::updateWeakHash32(WeakHash32 & hash) const
}
}
template <typename T>
void ColumnVector<T>::updateHashFast(SipHash & hash) const
{
hash.update(reinterpret_cast<const char *>(data.data()), size() * sizeof(data[0]));
}
template <typename T>
struct ColumnVector<T>::less
{

View File

@ -162,6 +162,8 @@ public:
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
size_t byteSize() const override
{
return data.size() * sizeof(data[0]);

View File

@ -216,6 +216,9 @@ public:
/// WeakHash32 must have the same size as column.
virtual void updateWeakHash32(WeakHash32 & hash) const = 0;
/// Update state of hash with all column.
virtual void updateHashFast(SipHash & hash) const = 0;
/** Removes elements that don't match the filter.
* Is used in WHERE and HAVING operations.
* If result_size_hint > 0, then makes advance reserve(result_size_hint) for the result column;

View File

@ -72,6 +72,10 @@ public:
{
}
void updateHashFast(SipHash & /*hash*/) const override
{
}
void insertFrom(const IColumn &, size_t) override
{
++s;

View File

@ -143,6 +143,11 @@ public:
throw Exception("Method updateWeakHash32 is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
}
void updateHashFast(SipHash &) const override
{
throw Exception("Method updateHashFast is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
}
void compareColumn(const IColumn &, size_t, PaddedPODArray<UInt64> *, PaddedPODArray<Int8> &, int, int) const override
{
throw Exception("Method compareColumn is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);

View File

@ -96,6 +96,8 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \
\
M(SettingMilliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \
\
M(SettingUInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \
M(SettingUInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \
\

View File

@ -1,7 +1,11 @@
#include <Storages/MergeTree/DataPartsExchange.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Disks/createVolume.h>
#include <Disks/SingleDiskVolume.h>
#include <Common/CurrentMetrics.h>
#include <Common/NetException.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <IO/HTTPCommon.h>
#include <ext/scope_guard.h>
#include <Poco/File.h>
@ -27,6 +31,8 @@ namespace ErrorCodes
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int CHECKSUM_DOESNT_MATCH;
extern const int INSECURE_PATH;
extern const int CORRUPTED_DATA;
extern const int LOGICAL_ERROR;
}
namespace DataPartsExchange
@ -36,6 +42,7 @@ namespace
{
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE = 1;
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS = 2;
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE = 3;
std::string getEndpointId(const std::string & node_id)
@ -76,7 +83,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
}
/// We pretend to work as older server version, to be sure that client will correctly process our version
response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS))});
response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE))});
++total_sends;
SCOPE_EXIT({--total_sends;});
@ -92,16 +99,8 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend};
/// We'll take a list of files from the list of checksums.
MergeTreeData::DataPart::Checksums checksums = part->checksums;
/// Add files that are not in the checksum list.
checksums.files["checksums.txt"];
checksums.files["columns.txt"];
MergeTreeData::DataPart::Checksums data_checksums;
if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE)
writeBinary(checksums.getTotalSizeOnDisk(), out);
writeBinary(part->checksums.getTotalSizeOnDisk(), out);
if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS)
{
@ -110,37 +109,13 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
writeBinary(ttl_infos_buffer.str(), out);
}
writeBinary(checksums.files.size(), out);
for (const auto & it : checksums.files)
{
String file_name = it.first;
if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE)
writeStringBinary(part->getType().toString(), out);
auto disk = part->volume->getDisk();
String path = part->getFullRelativePath() + file_name;
UInt64 size = disk->getFileSize(path);
writeStringBinary(it.first, out);
writeBinary(size, out);
auto file_in = disk->readFile(path);
HashingWriteBuffer hashing_out(out);
copyData(*file_in, hashing_out, blocker.getCounter());
if (blocker.isCancelled())
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
if (hashing_out.count() != size)
throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
writePODBinary(hashing_out.getHash(), out);
if (file_name != "checksums.txt" &&
file_name != "columns.txt")
data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash());
}
part->checksums.checkEqual(data_checksums, false);
if (isInMemoryPart(part))
sendPartFromMemory(part, out);
else
sendPartFromDisk(part, out);
}
catch (const NetException &)
{
@ -160,6 +135,61 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
}
}
void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out)
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
auto part_in_memory = asInMemoryPart(part);
if (!part_in_memory)
throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR);
NativeBlockOutputStream block_out(out, 0, metadata_snapshot->getSampleBlock());
part->checksums.write(out);
block_out.write(part_in_memory->block);
}
void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out)
{
/// We'll take a list of files from the list of checksums.
MergeTreeData::DataPart::Checksums checksums = part->checksums;
/// Add files that are not in the checksum list.
checksums.files["checksums.txt"] = {};
checksums.files["columns.txt"] = {};
MergeTreeData::DataPart::Checksums data_checksums;
writeBinary(checksums.files.size(), out);
for (const auto & it : checksums.files)
{
String file_name = it.first;
auto disk = part->volume->getDisk();
String path = part->getFullRelativePath() + file_name;
UInt64 size = disk->getFileSize(path);
writeStringBinary(it.first, out);
writeBinary(size, out);
auto file_in = disk->readFile(path);
HashingWriteBuffer hashing_out(out);
copyData(*file_in, hashing_out, blocker.getCounter());
if (blocker.isCancelled())
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
if (hashing_out.count() != size)
throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
writePODBinary(hashing_out.getHash(), out);
if (file_name != "checksums.txt" &&
file_name != "columns.txt")
data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash());
}
part->checksums.checkEqual(data_checksums, false);
}
MergeTreeData::DataPartPtr Service::findPart(const String & name)
{
/// It is important to include PreCommitted and Outdated parts here because remote replicas cannot reliably
@ -173,6 +203,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name)
}
MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
const StorageMetadataPtr & metadata_snapshot,
const String & part_name,
const String & replica_path,
const String & host,
@ -199,7 +230,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
{
{"endpoint", getEndpointId(replica_path)},
{"part", part_name},
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS)},
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE)},
{"compress", "false"}
});
@ -247,10 +278,46 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
reservation = data.makeEmptyReservationOnLargestDisk();
}
return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in);
String part_type = "Wide";
if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE)
readStringBinary(part_type, in);
return part_type == "InMemory" ? downloadPartToMemory(part_name, metadata_snapshot, std::move(reservation), in)
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in);
}
MergeTreeData::MutableDataPartPtr Fetcher::downloadPart(
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
const String & part_name,
const StorageMetadataPtr & metadata_snapshot,
ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in)
{
MergeTreeData::DataPart::Checksums checksums;
if (!checksums.read(in))
throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA);
NativeBlockInputStream block_in(in, 0);
auto block = block_in.read();
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, reservation->getDisk());
MergeTreeData::MutableDataPartPtr new_data_part =
std::make_shared<MergeTreeDataPartInMemory>(data, part_name, volume);
new_data_part->is_temp = true;
new_data_part->setColumns(block.getNamesAndTypesList());
new_data_part->minmax_idx.update(block, data.minmax_idx_columns);
new_data_part->partition.create(metadata_snapshot, block, 0);
MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, nullptr);
part_out.writePrefix();
part_out.write(block);
part_out.writeSuffixAndFinalizePart(new_data_part);
new_data_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true);
return new_data_part;
}
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
const String & part_name,
const String & replica_path,
bool to_detached,

View File

@ -31,6 +31,8 @@ public:
private:
MergeTreeData::DataPartPtr findPart(const String & name);
void sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
void sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
private:
/// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish,
@ -51,6 +53,7 @@ public:
/// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory.
MergeTreeData::MutableDataPartPtr fetchPart(
const StorageMetadataPtr & metadata_snapshot,
const String & part_name,
const String & replica_path,
const String & host,
@ -66,7 +69,7 @@ public:
ActionBlocker blocker;
private:
MergeTreeData::MutableDataPartPtr downloadPart(
MergeTreeData::MutableDataPartPtr downloadPartToDisk(
const String & part_name,
const String & replica_path,
bool to_detached,
@ -74,6 +77,12 @@ private:
const ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in);
MergeTreeData::MutableDataPartPtr downloadPartToMemory(
const String & part_name,
const StorageMetadataPtr & metadata_snapshot,
ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in);
MergeTreeData & data;
Poco::Logger * log;
};

View File

@ -386,8 +386,6 @@ String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize(const StorageM
String IMergeTreeDataPart::getFullPath() const
{
assertOnDisk();
if (relative_path.empty())
throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
@ -396,8 +394,6 @@ String IMergeTreeDataPart::getFullPath() const
String IMergeTreeDataPart::getFullRelativePath() const
{
assertOnDisk();
if (relative_path.empty())
throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
@ -778,12 +774,8 @@ void IMergeTreeDataPart::remove() const
}
}
String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const
String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const
{
/// Do not allow underscores in the prefix because they are used as separators.
assert(prefix.find_first_of('_') == String::npos);
String res;
/** If you need to detach a part, and directory into which we want to rename it already exists,
@ -793,7 +785,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix)
*/
for (int try_no = 0; try_no < 10; try_no++)
{
res = "detached/" + (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : "");
res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : "");
if (!volume->getDisk()->exists(getFullRelativePath() + res))
return res;
@ -804,17 +796,20 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix)
return res;
}
void IMergeTreeDataPart::renameToDetached(const String & prefix) const
String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const
{
assertOnDisk();
renameTo(getRelativePathForDetachedPart(prefix));
/// Do not allow underscores in the prefix because they are used as separators.
assert(prefix.find_first_of('_') == String::npos);
return "detached/" + getRelativePathForPrefix(prefix);
}
void IMergeTreeDataPart::makeCloneInDetached(const String & prefix) const
void IMergeTreeDataPart::renameToDetached(const String & prefix) const
{
assertOnDisk();
LOG_INFO(storage.log, "Detaching {}", relative_path);
renameTo(getRelativePathForDetachedPart(prefix), true);
}
void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const
{
String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix);
/// Backup is not recursive (max_level is 0), so do not copy inner directories
@ -897,13 +892,18 @@ void IMergeTreeDataPart::checkConsistencyBase() const
}
}
void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const
{
throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
}
void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
{
if (getColumns().empty() || checksums.empty())
throw Exception("Cannot calculate columns sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR);
calculateEachColumnSizesOnDisk(columns_sizes, total_columns_size);
calculateEachColumnSizes(columns_sizes, total_columns_size);
}
ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const
@ -932,4 +932,9 @@ bool isWidePart(const MergeTreeDataPartPtr & data_part)
return (data_part && data_part->getType() == MergeTreeDataPartType::WIDE);
}
bool isInMemoryPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::IN_MEMORY);
}
}

View File

@ -302,15 +302,15 @@ public:
/// Returns full path to part dir
String getFullPath() const;
/// Makes checks and move part to new directory
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = true) const;
/// Moves a part to detached/ directory and adds prefix to its name
void renameToDetached(const String & prefix) const;
/// Makes checks and move part to new directory
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const;
/// Makes clone of a part in detached/ directory via hard links
void makeCloneInDetached(const String & prefix) const;
virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const;
/// Makes full clone of part in detached/ on another disk
void makeCloneOnDiskDetached(const ReservationPtr & reservation) const;
@ -326,6 +326,8 @@ public:
static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from);
void calculateColumnsSizesOnDisk();
String getRelativePathForPrefix(const String & prefix) const;
protected:
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
ColumnSize total_columns_size;
@ -343,12 +345,14 @@ protected:
void removeIfNeeded();
virtual void checkConsistency(bool require_part_metadata) const = 0;
virtual void checkConsistency(bool require_part_metadata) const;
void checkConsistencyBase() const;
/// Fill each_columns_size and total_size with sizes from columns files on
/// disk using columns and checksums.
virtual void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0;
virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0;
String getRelativePathForDetachedPart(const String & prefix) const;
private:
/// In compact parts order of columns is necessary
@ -374,15 +378,14 @@ private:
void loadTTLInfos();
void loadPartitionAndMinMaxIndex();
/// Generate unique path to detach part
String getRelativePathForDetachedPart(const String & prefix) const;
};
using MergeTreeDataPartState = IMergeTreeDataPart::State;
using MergeTreeDataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
using MergeTreeMutableDataPartPtr = std::shared_ptr<IMergeTreeDataPart>;
bool isCompactPart(const MergeTreeDataPartPtr & data_part);
bool isWidePart(const MergeTreeDataPartPtr & data_part);
bool isInMemoryPart(const MergeTreeDataPartPtr & data_part);
}

View File

@ -1,358 +1,41 @@
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <utility>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
constexpr auto INDEX_FILE_EXTENSION = ".idx";
}
void IMergeTreeDataPartWriter::Stream::finalize()
{
compressed.next();
plain_file->next();
marks.next();
}
void IMergeTreeDataPartWriter::Stream::sync() const
{
plain_file->sync();
marks_file->sync();
}
IMergeTreeDataPartWriter::Stream::Stream(
const String & escaped_column_name_,
DiskPtr disk_,
const String & data_path_,
const std::string & data_file_extension_,
const std::string & marks_path_,
const std::string & marks_file_extension_,
const CompressionCodecPtr & compression_codec_,
size_t max_compress_block_size_,
size_t estimated_size_,
size_t aio_threshold_) :
escaped_column_name(escaped_column_name_),
data_file_extension{data_file_extension_},
marks_file_extension{marks_file_extension_},
plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite, estimated_size_, aio_threshold_)),
plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_codec_), compressed(compressed_buf),
marks_file(disk_->writeFile(marks_path_ + marks_file_extension, 4096, WriteMode::Rewrite)), marks(*marks_file)
{
}
void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums)
{
String name = escaped_column_name;
checksums.files[name + data_file_extension].is_compressed = true;
checksums.files[name + data_file_extension].uncompressed_size = compressed.count();
checksums.files[name + data_file_extension].uncompressed_hash = compressed.getHash();
checksums.files[name + data_file_extension].file_size = plain_hashing.count();
checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash();
checksums.files[name + marks_file_extension].file_size = marks.count();
checksums.files[name + marks_file_extension].file_hash = marks.getHash();
}
IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_)
const MergeTreeWriterSettings & settings_)
: data_part(data_part_)
, part_path(data_part_->getFullRelativePath())
, storage(data_part_->storage)
, metadata_snapshot(metadata_snapshot_)
, columns_list(columns_list_)
, marks_file_extension(marks_file_extension_)
, index_granularity(index_granularity_)
, default_codec(default_codec_)
, skip_indices(indices_to_recalc_)
, settings(settings_)
, compute_granularity(index_granularity.empty())
, with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity)
, with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity){}
IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const MergeTreeIndices & skip_indices_,
const MergeTreeIndexGranularity & index_granularity_,
const MergeTreeWriterSettings & settings_)
: data_part(data_part_)
, storage(data_part_->storage)
, metadata_snapshot(metadata_snapshot_)
, columns_list(columns_list_)
, skip_indices(skip_indices_)
, index_granularity(index_granularity_)
, settings(settings_)
, with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity) {}
Columns IMergeTreeDataPartWriter::releaseIndexColumns()
{
if (settings.blocks_are_granules_size && !index_granularity.empty())
throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR);
auto disk = data_part->volume->getDisk();
if (!disk->exists(part_path))
disk->createDirectories(part_path);
}
IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default;
/// Implemetation is splitted into static functions for ability
/// of making unit tests without creation instance of IMergeTreeDataPartWriter,
/// which requires a lot of dependencies and access to filesystem.
static size_t computeIndexGranularityImpl(
const Block & block,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity)
{
size_t rows_in_block = block.rows();
size_t index_granularity_for_block;
if (!can_use_adaptive_index_granularity)
index_granularity_for_block = fixed_index_granularity_rows;
else
{
size_t block_size_in_memory = block.bytes();
if (blocks_are_granules)
index_granularity_for_block = rows_in_block;
else if (block_size_in_memory >= index_granularity_bytes)
{
size_t granules_in_block = block_size_in_memory / index_granularity_bytes;
index_granularity_for_block = rows_in_block / granules_in_block;
}
else
{
size_t size_of_row_in_bytes = block_size_in_memory / rows_in_block;
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes;
}
}
if (index_granularity_for_block == 0) /// very rare case when index granularity bytes less then single row
index_granularity_for_block = 1;
/// We should be less or equal than fixed index granularity
index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block);
return index_granularity_for_block;
}
static void fillIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity,
size_t index_offset,
size_t index_granularity_for_block,
size_t rows_in_block)
{
for (size_t current_row = index_offset; current_row < rows_in_block; current_row += index_granularity_for_block)
index_granularity.appendMark(index_granularity_for_block);
}
size_t IMergeTreeDataPartWriter::computeIndexGranularity(const Block & block)
{
const auto storage_settings = storage.getSettings();
return computeIndexGranularityImpl(
block,
storage_settings->index_granularity_bytes,
storage_settings->index_granularity,
settings.blocks_are_granules_size,
settings.can_use_adaptive_granularity);
}
void IMergeTreeDataPartWriter::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block)
{
fillIndexGranularityImpl(
index_granularity,
index_offset,
index_granularity_for_block,
rows_in_block);
}
void IMergeTreeDataPartWriter::initPrimaryIndex()
{
if (metadata_snapshot->hasPrimaryKey())
{
index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
}
primary_index_initialized = true;
}
void IMergeTreeDataPartWriter::initSkipIndices()
{
for (const auto & index_helper : skip_indices)
{
String stream_name = index_helper->getFileName();
skip_indices_streams.emplace_back(
std::make_unique<IMergeTreeDataPartWriter::Stream>(
stream_name,
data_part->volume->getDisk(),
part_path + stream_name, INDEX_FILE_EXTENSION,
part_path + stream_name, marks_file_extension,
default_codec, settings.max_compress_block_size,
0, settings.aio_threshold));
skip_indices_aggregators.push_back(index_helper->createIndexAggregator());
skip_index_filling.push_back(0);
}
skip_indices_initialized = true;
}
void IMergeTreeDataPartWriter::calculateAndSerializePrimaryIndex(const Block & primary_index_block, size_t rows)
{
if (!primary_index_initialized)
throw Exception("Primary index is not initialized", ErrorCodes::LOGICAL_ERROR);
size_t primary_columns_num = primary_index_block.columns();
if (index_columns.empty())
{
index_types = primary_index_block.getDataTypes();
index_columns.resize(primary_columns_num);
last_index_row.resize(primary_columns_num);
for (size_t i = 0; i < primary_columns_num; ++i)
index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty();
}
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
/// Write index. The index contains Primary Key value for each `index_granularity` row.
size_t current_row = index_offset;
size_t total_marks = index_granularity.getMarksCount();
while (index_mark < total_marks && current_row < rows)
{
if (metadata_snapshot->hasPrimaryKey())
{
for (size_t j = 0; j < primary_columns_num; ++j)
{
const auto & primary_column = primary_index_block.getByPosition(j);
index_columns[j]->insertFrom(*primary_column.column, current_row);
primary_column.type->serializeBinary(*primary_column.column, current_row, *index_stream);
}
}
current_row += index_granularity.getMarkRows(index_mark++);
}
/// store last index row to write final mark at the end of column
for (size_t j = 0; j < primary_columns_num; ++j)
{
const IColumn & primary_column = *primary_index_block.getByPosition(j).column.get();
primary_column.get(rows - 1, last_index_row[j]);
}
}
void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices(
const Block & skip_indexes_block, size_t rows)
{
if (!skip_indices_initialized)
throw Exception("Skip indices are not initialized", ErrorCodes::LOGICAL_ERROR);
size_t skip_index_current_data_mark = 0;
/// Filling and writing skip indices like in MergeTreeDataPartWriterWide::writeColumn
for (size_t i = 0; i < skip_indices.size(); ++i)
{
const auto index_helper = skip_indices[i];
auto & stream = *skip_indices_streams[i];
size_t prev_pos = 0;
skip_index_current_data_mark = skip_index_data_mark;
while (prev_pos < rows)
{
UInt64 limit = 0;
if (prev_pos == 0 && index_offset != 0)
{
limit = index_offset;
}
else
{
limit = index_granularity.getMarkRows(skip_index_current_data_mark);
if (skip_indices_aggregators[i]->empty())
{
skip_indices_aggregators[i] = index_helper->createIndexAggregator();
skip_index_filling[i] = 0;
if (stream.compressed.offset() >= settings.min_compress_block_size)
stream.compressed.next();
writeIntBinary(stream.plain_hashing.count(), stream.marks);
writeIntBinary(stream.compressed.offset(), stream.marks);
/// Actually this numbers is redundant, but we have to store them
/// to be compatible with normal .mrk2 file format
if (settings.can_use_adaptive_granularity)
writeIntBinary(1UL, stream.marks);
}
/// this mark is aggregated, go to the next one
skip_index_current_data_mark++;
}
size_t pos = prev_pos;
skip_indices_aggregators[i]->update(skip_indexes_block, &pos, limit);
if (pos == prev_pos + limit)
{
++skip_index_filling[i];
/// write index if it is filled
if (skip_index_filling[i] == index_helper->index.granularity)
{
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
skip_index_filling[i] = 0;
}
}
prev_pos = pos;
}
}
skip_index_data_mark = skip_index_current_data_mark;
}
void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums)
{
bool write_final_mark = (with_final_mark && data_written);
if (write_final_mark && compute_granularity)
index_granularity.appendMark(0);
if (index_stream)
{
if (write_final_mark)
{
for (size_t j = 0; j < index_columns.size(); ++j)
{
index_columns[j]->insert(last_index_row[j]);
index_types[j]->serializeBinary(last_index_row[j], *index_stream);
}
last_index_row.clear();
}
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
index_stream = nullptr;
}
}
void IMergeTreeDataPartWriter::finishSkipIndicesSerialization(
MergeTreeData::DataPart::Checksums & checksums)
{
for (size_t i = 0; i < skip_indices.size(); ++i)
{
auto & stream = *skip_indices_streams[i];
if (!skip_indices_aggregators[i]->empty())
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
}
for (auto & stream : skip_indices_streams)
{
stream->finalize();
stream->addToChecksums(checksums);
}
skip_indices_streams.clear();
skip_indices_aggregators.clear();
skip_index_filling.clear();
return Columns(
std::make_move_iterator(index_columns.begin()),
std::make_move_iterator(index_columns.end()));
}
void IMergeTreeDataPartWriter::next()
@ -361,4 +44,6 @@ void IMergeTreeDataPartWriter::next()
index_offset = next_index_offset;
}
IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default;
}

View File

@ -19,56 +19,19 @@ namespace DB
class IMergeTreeDataPartWriter : private boost::noncopyable
{
public:
using WrittenOffsetColumns = std::set<std::string>;
/// Helper class, which holds chain of buffers to write data file with marks.
/// It is used to write: one column, skip index or all columns (in compact format).
struct Stream
{
Stream(
const String & escaped_column_name_,
DiskPtr disk_,
const String & data_path_,
const std::string & data_file_extension_,
const std::string & marks_path_,
const std::string & marks_file_extension_,
const CompressionCodecPtr & compression_codec_,
size_t max_compress_block_size_,
size_t estimated_size_,
size_t aio_threshold_);
String escaped_column_name;
std::string data_file_extension;
std::string marks_file_extension;
/// compressed -> compressed_buf -> plain_hashing -> plain_file
std::unique_ptr<WriteBufferFromFileBase> plain_file;
HashingWriteBuffer plain_hashing;
CompressedWriteBuffer compressed_buf;
HashingWriteBuffer compressed;
/// marks -> marks_file
std::unique_ptr<WriteBufferFromFileBase> marks_file;
HashingWriteBuffer marks;
void finalize();
void sync() const;
void addToChecksums(IMergeTreeDataPart::Checksums & checksums);
};
using StreamPtr = std::unique_ptr<Stream>;
IMergeTreeDataPartWriter(
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const MergeTreeWriterSettings & settings_);
IMergeTreeDataPartWriter(
const MergeTreeData::DataPartPtr & data_part,
const NamesAndTypesList & columns_list,
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const String & marks_file_extension,
const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings,
const MergeTreeIndexGranularity & index_granularity);
const MergeTreeIndices & skip_indices_,
const MergeTreeIndexGranularity & index_granularity_,
const MergeTreeWriterSettings & settings_);
virtual ~IMergeTreeDataPartWriter();
@ -77,40 +40,26 @@ public:
/* Blocks with already sorted index columns */
const Block & primary_key_block = {}, const Block & skip_indexes_block = {}) = 0;
void calculateAndSerializePrimaryIndex(const Block & primary_index_block, size_t rows);
void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, size_t rows);
virtual void calculateAndSerializePrimaryIndex(const Block & /* primary_index_block */) {}
virtual void calculateAndSerializeSkipIndices(const Block & /* skip_indexes_block */) {}
/// Shift mark and offset to prepare read next mark.
/// You must call it after calling write method and optionally
/// calling calculations of primary and skip indices.
void next();
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; }
Columns releaseIndexColumns()
{
return Columns(std::make_move_iterator(index_columns.begin()), std::make_move_iterator(index_columns.end()));
}
void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_)
{
written_offset_columns = written_offset_columns_;
}
const MergeTreeIndices & getSkipIndices() { return skip_indices; }
void initSkipIndices();
void initPrimaryIndex();
virtual void initSkipIndices() {}
virtual void initPrimaryIndex() {}
virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0;
void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums);
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums);
virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {}
virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {}
Columns releaseIndexColumns();
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; }
const MergeTreeIndices & getSkipIndices() { return skip_indices; }
protected:
/// Count index_granularity for block and store in `index_granularity`
size_t computeIndexGranularity(const Block & block);
virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block);
size_t getCurrentMark() const { return current_mark; }
size_t getIndexOffset() const { return index_offset; }
@ -118,56 +67,24 @@ protected:
using SerializationStates = std::unordered_map<String, SerializationState>;
MergeTreeData::DataPartPtr data_part;
String part_path;
const MergeTreeData & storage;
StorageMetadataPtr metadata_snapshot;
NamesAndTypesList columns_list;
const String marks_file_extension;
MergeTreeIndexGranularity index_granularity;
CompressionCodecPtr default_codec;
MergeTreeIndices skip_indices;
MergeTreeIndexGranularity index_granularity;
MergeTreeWriterSettings settings;
bool compute_granularity;
bool with_final_mark;
size_t next_mark = 0;
size_t next_index_offset = 0;
/// Number of marks in data from which skip indices have to start
/// aggregation. I.e. it's data mark number, not skip indices mark.
size_t skip_index_data_mark = 0;
std::vector<StreamPtr> skip_indices_streams;
MergeTreeIndexAggregators skip_indices_aggregators;
std::vector<size_t> skip_index_filling;
std::unique_ptr<WriteBufferFromFileBase> index_file_stream;
std::unique_ptr<HashingWriteBuffer> index_stream;
MutableColumns index_columns;
DataTypes index_types;
/// Index columns values from the last row from the last block
/// It's written to index file in the `writeSuffixAndFinalizePart` method
Row last_index_row;
bool data_written = false;
bool primary_index_initialized = false;
bool skip_indices_initialized = false;
/// To correctly write Nested elements column-by-column.
WrittenOffsetColumns * written_offset_columns = nullptr;
private:
/// Data is already written up to this mark.
size_t current_mark = 0;
/// The offset to the first row of the block for which you want to write the index.
size_t index_offset = 0;
/// Index is already serialized up to this mark.
size_t index_mark = 0;
};
}

View File

@ -258,4 +258,28 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns)
}
}
IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const String & column_name) const
{
String table_name = Nested::extractTableName(column_name);
for (const auto & part_column : data_part->getColumns())
{
if (typeid_cast<const DataTypeArray *>(part_column.type.get()))
{
auto position = data_part->getColumnPosition(part_column.name);
if (position && Nested::extractTableName(part_column.name) == table_name)
return position;
}
}
return {};
}
void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const
{
if (num_columns_to_read != columns.size())
throw Exception("invalid number of columns passed to MergeTreeReader::readRows. "
"Expected " + toString(columns.size()) + ", "
"got " + toString(num_columns_to_read), ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -63,6 +63,8 @@ protected:
/// Returns actual column type in part, which can differ from table metadata.
NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const;
void checkNumberOfColumns(size_t num_columns_to_read) const;
/// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size.
ValueSizeMap avg_value_size_hints;
/// Stores states for IDataType::deserializeBinaryBulk
@ -80,6 +82,9 @@ protected:
StorageMetadataPtr metadata_snapshot;
MarkRanges all_mark_ranges;
using ColumnPosition = std::optional<size_t>;
ColumnPosition findColumnForOffsets(const String & column_name) const;
friend class MergeTreeRangeReader::DelayedStream;
private:

View File

@ -11,7 +11,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream(
: storage(data_part->storage)
, metadata_snapshot(metadata_snapshot_)
, volume(data_part->volume)
, part_path(data_part->getFullRelativePath())
, part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "")
{
}

View File

@ -60,7 +60,7 @@ public:
const Partitions & partitions,
const size_t max_total_size_to_merge) = 0;
virtual ~IMergeSelector() {}
virtual ~IMergeSelector() = default;
};
}

View File

@ -1,4 +1,5 @@
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/StorageMergeTree.h>
#include <Interpreters/PartLog.h>
@ -26,9 +27,11 @@ void MergeTreeBlockOutputStream::write(const Block & block)
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
if (storage.merging_mutating_task_handle)
{
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.merging_mutating_task_handle->signalReadyToRun();
}
}
}

View File

@ -14,10 +14,7 @@ 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_)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, max_parts_per_block(max_parts_per_block_)

View File

@ -28,6 +28,7 @@
#include <Parsers/queryToString.h>
#include <Storages/AlterCommands.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
@ -234,7 +235,8 @@ MergeTreeData::MergeTreeData(
String reason;
if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty())
LOG_WARNING(log, "{} Settings 'min_bytes_for_wide_part' and 'min_bytes_for_wide_part' will be ignored.", reason);
LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', "
"'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason);
}
StoragePolicyPtr MergeTreeData::getStoragePolicy() const
@ -671,8 +673,10 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
{
LOG_DEBUG(log, "Loading data parts");
auto metadata_snapshot = getInMemoryMetadataPtr();
const auto settings = getSettings();
std::vector<std::pair<String, DiskPtr>> part_names_with_disks;
MutableDataPartsVector parts_from_wal;
Strings part_file_names;
auto disks = getStoragePolicy()->getDisks();
@ -711,13 +715,27 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
continue;
part_names_with_disks.emplace_back(it->name(), disk_ptr);
/// Create and correctly initialize global WAL object, if it's needed
if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal)
{
write_ahead_log = std::make_shared<MergeTreeWriteAheadLog>(*this, disk_ptr, it->name());
for (auto && part : write_ahead_log->restore(metadata_snapshot))
parts_from_wal.push_back(std::move(part));
}
else if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME))
{
MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name());
for (auto && part : wal.restore(metadata_snapshot))
parts_from_wal.push_back(std::move(part));
}
}
}
auto part_lock = lockParts();
data_parts_indexes.clear();
if (part_names_with_disks.empty())
if (part_names_with_disks.empty() && parts_from_wal.empty())
{
LOG_DEBUG(log, "There is no data parts");
return;
@ -853,6 +871,19 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
pool.wait();
for (auto & part : parts_from_wal)
{
if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock))
continue;
part->modification_time = time(nullptr);
/// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later
part->state = DataPartState::Committed;
if (!data_parts_indexes.insert(part).second)
throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART);
}
if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts)
throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR);
@ -999,7 +1030,8 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force)
if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example).
((part_remove_time < now &&
now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force))
now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force
|| isInMemoryPart(part))) /// Remove in-memory parts immediatly to not store excessive data in RAM
{
parts_to_delete.emplace_back(it);
}
@ -1115,6 +1147,61 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
}
}
void MergeTreeData::clearOldWriteAheadLogs()
{
DataPartsVector parts = getDataPartsVector();
std::vector<std::pair<Int64, Int64>> all_block_numbers_on_disk;
std::vector<std::pair<Int64, Int64>> block_numbers_on_disk;
for (const auto & part : parts)
if (part->isStoredOnDisk())
all_block_numbers_on_disk.emplace_back(part->info.min_block, part->info.max_block);
if (all_block_numbers_on_disk.empty())
return;
std::sort(all_block_numbers_on_disk.begin(), all_block_numbers_on_disk.end());
block_numbers_on_disk.push_back(all_block_numbers_on_disk[0]);
for (size_t i = 1; i < all_block_numbers_on_disk.size(); ++i)
{
if (all_block_numbers_on_disk[i].first == all_block_numbers_on_disk[i - 1].second + 1)
block_numbers_on_disk.back().second = all_block_numbers_on_disk[i].second;
else
block_numbers_on_disk.push_back(all_block_numbers_on_disk[i]);
}
auto is_range_on_disk = [&block_numbers_on_disk](Int64 min_block, Int64 max_block)
{
auto lower = std::lower_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, Int64(-1L)));
if (lower != block_numbers_on_disk.end() && min_block >= lower->first && max_block <= lower->second)
return true;
if (lower != block_numbers_on_disk.begin())
{
--lower;
if (min_block >= lower->first && max_block <= lower->second)
return true;
}
return false;
};
auto disks = getStoragePolicy()->getDisks();
for (auto disk_it = disks.rbegin(); disk_it != disks.rend(); ++disk_it)
{
auto disk_ptr = *disk_it;
for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next())
{
auto min_max_block_number = MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(it->name());
if (min_max_block_number && is_range_on_disk(min_max_block_number->first, min_max_block_number->second))
{
LOG_DEBUG(log, "Removing from filesystem outdated WAL file " + it->name());
disk_ptr->remove(relative_data_path + it->name());
}
}
}
}
void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id)
{
auto disks = getStoragePolicy()->getDisks();
@ -1445,10 +1532,25 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const
{
if (!canUseAdaptiveGranularity())
const auto settings = getSettings();
if (!canUsePolymorphicParts(*settings))
return MergeTreeDataPartType::WIDE;
if (bytes_uncompressed < settings->min_bytes_for_compact_part || rows_count < settings->min_rows_for_compact_part)
return MergeTreeDataPartType::IN_MEMORY;
if (bytes_uncompressed < settings->min_bytes_for_wide_part || rows_count < settings->min_rows_for_wide_part)
return MergeTreeDataPartType::COMPACT;
return MergeTreeDataPartType::WIDE;
}
MergeTreeDataPartType MergeTreeData::choosePartTypeOnDisk(size_t bytes_uncompressed, size_t rows_count) const
{
const auto settings = getSettings();
if (!canUsePolymorphicParts(*settings))
return MergeTreeDataPartType::WIDE;
if (bytes_uncompressed < settings->min_bytes_for_wide_part || rows_count < settings->min_rows_for_wide_part)
return MergeTreeDataPartType::COMPACT;
@ -1464,8 +1566,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(const String & name,
return std::make_shared<MergeTreeDataPartCompact>(*this, name, part_info, volume, relative_path);
else if (type == MergeTreeDataPartType::WIDE)
return std::make_shared<MergeTreeDataPartWide>(*this, name, part_info, volume, relative_path);
else if (type == MergeTreeDataPartType::IN_MEMORY)
return std::make_shared<MergeTreeDataPartInMemory>(*this, name, part_info, volume, relative_path);
else
throw Exception("Unknown type in part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE);
throw Exception("Unknown type of part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE);
}
static MergeTreeDataPartType getPartTypeFromMarkExtension(const String & mrk_ext)
@ -1499,7 +1603,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
else
{
/// Didn't find any mark file, suppose that part is empty.
type = choosePartType(0, 0);
type = choosePartTypeOnDisk(0, 0);
}
return createPart(name, type, part_info, volume, relative_path);
@ -1563,9 +1667,9 @@ void MergeTreeData::changeSettings(
}
}
void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableLockHolder &)
void MergeTreeData::freezeAll(const String & with_name, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableLockHolder &)
{
freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context);
freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, metadata_snapshot, with_name, context);
}
void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name)
@ -1752,6 +1856,7 @@ void MergeTreeData::renameTempPartAndReplace(
DataPartPtr covering_part;
DataPartsVector covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock);
DataPartsVector covered_parts_in_memory;
if (covering_part)
{
@ -1767,7 +1872,7 @@ void MergeTreeData::renameTempPartAndReplace(
part->info = part_info;
part->is_temp = false;
part->state = DataPartState::PreCommitted;
part->renameTo(part_name);
part->renameTo(part_name, true);
auto part_it = data_parts_indexes.insert(part).first;
@ -1789,6 +1894,13 @@ void MergeTreeData::renameTempPartAndReplace(
addPartContributionToColumnSizes(part);
}
auto part_in_memory = asInMemoryPart(part);
if (part_in_memory && getSettings()->in_memory_parts_enable_wal)
{
auto wal = getWriteAheadLog();
wal->addPart(part_in_memory->block, part_in_memory->name);
}
if (out_covered_parts)
{
for (DataPartPtr & covered_part : covered_parts)
@ -1825,6 +1937,9 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect
if (part->state != IMergeTreeDataPart::State::Outdated)
modifyPartState(part, IMergeTreeDataPart::State::Outdated);
if (isInMemoryPart(part) && getSettings()->in_memory_parts_enable_wal)
getWriteAheadLog()->dropPart(part->name);
}
}
@ -2371,7 +2486,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
}
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableLockHolder &)
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder &)
{
std::optional<String> prefix;
String partition_id;
@ -2403,6 +2518,7 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String &
else
return part->info.partition_id == partition_id;
},
metadata_snapshot,
with_name,
context);
}
@ -3083,9 +3199,11 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(
return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot);
}
MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(const MergeTreeData::DataPartPtr & src_part,
const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info)
MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part,
const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info,
const StorageMetadataPtr & metadata_snapshot)
{
/// Check that the storage policy contains the disk where the src_part is located.
bool does_storage_policy_allow_same_disk = false;
@ -3112,6 +3230,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
if (disk->exists(dst_part_path))
throw Exception("Part in " + fullPath(disk, dst_part_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
/// If source part is in memory, flush it to disk and clone it already in on-disk format
if (auto src_part_in_memory = asInMemoryPart(src_part))
{
const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path;
auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix);
src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path, metadata_snapshot);
src_part_path = src_relative_data_path + flushed_part_path + "/";
}
LOG_DEBUG(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path));
localBackup(disk, src_part_path, dst_part_path);
disk->removeIfExists(dst_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH);
@ -3171,7 +3298,7 @@ MergeTreeData::PathsWithDisks MergeTreeData::getRelativeDataPathsWithDisks() con
return res;
}
void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context)
void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context)
{
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();
String default_shadow_path = clickhouse_path + "shadow/";
@ -3200,7 +3327,11 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
LOG_DEBUG(log, "Freezing part {} snapshot will be placed at {}", part->name, backup_path);
String backup_part_path = backup_path + relative_data_path + part->relative_path;
localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path);
if (auto part_in_memory = asInMemoryPart(part))
part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path, metadata_snapshot);
else
localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path);
part->volume->getDisk()->removeIfExists(backup_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH);
part->is_frozen.store(true, std::memory_order_relaxed);
@ -3434,11 +3565,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S
{
if (!canUseAdaptiveGranularity())
{
if ((settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0) && out_reason)
if (out_reason && (settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0
|| settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0))
{
std::ostringstream message;
message << "Table can't create parts with adaptive granularity, but settings min_rows_for_wide_part = "
<< settings.min_rows_for_wide_part << ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part
message << "Table can't create parts with adaptive granularity, but settings"
<< "min_rows_for_wide_part = " << settings.min_rows_for_wide_part
<< ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part
<< ", min_rows_for_compact_part = " << settings.min_rows_for_compact_part
<< ", min_bytes_for_compact_part = " << settings.min_bytes_for_compact_part
<< ". Parts with non-adaptive granularity can be stored only in Wide (default) format.";
*out_reason = message.str();
}
@ -3464,6 +3599,18 @@ MergeTreeData::AlterConversions MergeTreeData::getAlterConversionsForPart(const
return result;
}
MergeTreeData::WriteAheadLogPtr MergeTreeData::getWriteAheadLog()
{
std::lock_guard lock(write_ahead_log_mutex);
if (!write_ahead_log)
{
auto reservation = reserveSpace(getSettings()->write_ahead_log_max_bytes);
write_ahead_log = std::make_shared<MergeTreeWriteAheadLog>(*this, reservation->getDisk());
}
return write_ahead_log;
}
NamesAndTypesList MergeTreeData::getVirtuals() const
{
return NamesAndTypesList{
@ -3473,4 +3620,5 @@ NamesAndTypesList MergeTreeData::getVirtuals() const
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
};
}
}

View File

@ -16,8 +16,10 @@
#include <DataTypes/DataTypesNumber.h>
#include <Processors/Merges/Algorithms/Graphite.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/IndicesDescription.h>
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Storages/MergeTree/MergeTreeWriteAheadLog.h>
#include <Interpreters/PartLog.h>
#include <Disks/StoragePolicy.h>
#include <Interpreters/Aggregator.h>
@ -193,13 +195,15 @@ public:
DataPartsLock lockParts() const { return DataPartsLock(data_parts_mutex); }
MergeTreeDataPartType choosePartType(size_t bytes_uncompressed, size_t rows_count) const;
MergeTreeDataPartType choosePartTypeOnDisk(size_t bytes_uncompressed, size_t rows_count) const;
/// After this method setColumns must be called
MutableDataPartPtr createPart(const String & name,
MergeTreeDataPartType type, const MergeTreePartInfo & part_info,
const VolumePtr & volume, const String & relative_path) const;
/// After this methods 'loadColumnsChecksumsIndexes' must be called
/// Create part, that already exists on filesystem.
/// After this methods 'loadColumnsChecksumsIndexes' must be called.
MutableDataPartPtr createPart(const String & name,
const VolumePtr & volume, const String & relative_path) const;
@ -474,6 +478,9 @@ public:
void clearOldPartsFromFilesystem(bool force = false);
void clearPartsFromFilesystem(const DataPartsVector & parts);
/// Delete WAL files containing parts, that all already stored on disk.
void clearOldWriteAheadLogs();
/// Delete all directories which names begin with "tmp"
/// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime
/// Must be called with locked lockForShare() because use relative_data_path.
@ -504,7 +511,7 @@ public:
TableLockHolder & table_lock_holder);
/// Freezes all parts.
void freezeAll(const String & with_name, const Context & context, TableLockHolder & table_lock_holder);
void freezeAll(const String & with_name, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableLockHolder & table_lock_holder);
/// Should be called if part data is suspected to be corrupted.
void reportBrokenPart(const String & name) const
@ -526,7 +533,7 @@ public:
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
* or if 'with_name' is specified - backup is created in directory with specified name.
*/
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableLockHolder & table_lock_holder);
void freezePartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder & table_lock_holder);
public:
@ -559,7 +566,7 @@ public:
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info);
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot);
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;
@ -632,6 +639,9 @@ public:
/// TTL rule.
bool isPartInTTLDestination(const TTLDescription & ttl, const IMergeTreeDataPart & part) const;
using WriteAheadLogPtr = std::shared_ptr<MergeTreeWriteAheadLog>;
WriteAheadLogPtr getWriteAheadLog();
MergeTreeDataFormatVersion format_version;
Context & global_context;
@ -816,7 +826,7 @@ protected:
/// Common part for |freezePartition()| and |freezeAll()|.
using MatcherFn = std::function<bool(const DataPartPtr &)>;
void freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context);
void freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context);
bool canReplacePartition(const DataPartPtr & src_part) const;
@ -870,7 +880,10 @@ private:
/// Check selected parts for movements. Used by ALTER ... MOVE queries.
CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, SpacePtr space);
bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason) const;
bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const;
std::mutex write_ahead_log_mutex;
WriteAheadLogPtr write_ahead_log;
virtual void startBackgroundMovesIfNeeded() = 0;
};

View File

@ -87,7 +87,7 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_)
sum_bytes_uncompressed += part->getTotalColumnsSize().data_uncompressed;
}
auto future_part_type = parts_.front()->storage.choosePartType(sum_bytes_uncompressed, sum_rows);
auto future_part_type = parts_.front()->storage.choosePartTypeOnDisk(sum_bytes_uncompressed, sum_rows);
assign(std::move(parts_), future_part_type);
}
@ -231,12 +231,14 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
{
/// Check predicate only for first part in each partition.
if (!prev_part)
{
/* Parts can be merged with themselves for TTL needs for example.
* So we have to check if this part is currently being inserted with quorum and so on and so forth.
* Obviously we have to check it manually only for the first part
* of each partition because it will be automatically checked for a pair of parts. */
if (!can_merge_callback(nullptr, part, nullptr))
continue;
}
const String & partition_id = part->info.partition_id;
if (!prev_partition_id || partition_id != *prev_partition_id || (prev_part && !can_merge_callback(*prev_part, part, nullptr)))
@ -1021,10 +1023,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
commands_for_part.emplace_back(command);
}
if (!isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading))
if (source_part->isStoredOnDisk() && !isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading))
{
LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation);
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info);
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot);
}
else
{
@ -1090,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
need_remove_expired_values = true;
/// All columns from part are changed and may be some more that were missing before in part
if (isCompactPart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
if (!isWidePart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
{
auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames);
mutateAllPartColumns(
@ -1284,7 +1286,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
{
ColumnsDescription part_columns(part->getColumns());
if (isCompactPart(part))
if (!isWidePart(part))
{
NameSet mutated_columns;
for (const auto & command : commands)
@ -1472,7 +1474,7 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart(
{
/// In compact parts we read all columns, because they all stored in a
/// single file
if (isCompactPart(source_part))
if (!isWidePart(source_part))
return updated_header.getNamesAndTypesList();
NameSet removed_columns;
@ -1635,9 +1637,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
merge_entry->bytes_written_uncompressed += block.bytes();
}
new_data_part->minmax_idx = std::move(minmax_idx);
mutating_stream->readSuffix();
out.writeSuffixAndFinalizePart(new_data_part);
}

View File

@ -1,7 +1,6 @@
#include "MergeTreeDataPartCompact.h"
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
#include <Storages/MergeTree/IMergeTreeReader.h>
#include <Poco/File.h>
@ -76,7 +75,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
}
void MergeTreeDataPartCompact::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & total_size) const
void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*each_columns_size*/, ColumnSize & total_size) const
{
auto bin_checksum = checksums.files.find(DATA_FILE_NAME_WITH_EXTENSION);
if (bin_checksum != checksums.files.end())

View File

@ -68,7 +68,7 @@ private:
void loadIndexGranularity() override;
/// Compact parts doesn't support per column size, only total size
void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
};
}

View File

@ -0,0 +1,133 @@
#include "MergeTreeDataPartInMemory.h"
#include <Storages/MergeTree/MergeTreeReaderInMemory.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterInMemory.h>
#include <Storages/MergeTree/IMergeTreeReader.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int DIRECTORY_ALREADY_EXISTS;
}
MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
MergeTreeData & storage_,
const String & name_,
const VolumePtr & volume_,
const std::optional<String> & relative_path_)
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY)
{
}
MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
const MergeTreeData & storage_,
const String & name_,
const MergeTreePartInfo & info_,
const VolumePtr & volume_,
const std::optional<String> & relative_path_)
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY)
{
}
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader(
const NamesAndTypesList & columns_to_read,
const StorageMetadataPtr & metadata_snapshot,
const MarkRanges & mark_ranges,
UncompressedCache * /* uncompressed_cache */,
MarkCache * /* mark_cache */,
const MergeTreeReaderSettings & reader_settings,
const ValueSizeMap & /* avg_value_size_hints */,
const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const
{
auto ptr = std::static_pointer_cast<const MergeTreeDataPartInMemory>(shared_from_this());
return std::make_unique<MergeTreeReaderInMemory>(
ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings);
}
IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter(
const NamesAndTypesList & columns_list,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MergeTreeIndexPtr> & /* indices_to_recalc */,
const CompressionCodecPtr & /* default_codec */,
const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & /* computed_index_granularity */) const
{
auto ptr = std::static_pointer_cast<const MergeTreeDataPartInMemory>(shared_from_this());
return std::make_unique<MergeTreeDataPartWriterInMemory>(
ptr, columns_list, metadata_snapshot, writer_settings);
}
void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const
{
const auto & disk = volume->getDisk();
String destination_path = base_path + new_relative_path;
auto new_type = storage.choosePartTypeOnDisk(block.bytes(), rows_count);
auto new_data_part = storage.createPart(name, new_type, info, volume, new_relative_path);
new_data_part->setColumns(columns);
new_data_part->partition.value.assign(partition.value);
new_data_part->minmax_idx = minmax_idx;
if (disk->exists(destination_path))
{
throw Exception("Could not flush part " + quoteString(getFullPath())
+ ". Part in " + fullPath(disk, destination_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
}
disk->createDirectories(destination_path);
auto compression_codec = storage.global_context.chooseCompressionCodec(0, 0);
auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices());
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec);
out.writePrefix();
out.write(block);
out.writeSuffixAndFinalizePart(new_data_part);
}
void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const
{
String detached_path = getRelativePathForDetachedPart(prefix);
flushToDisk(storage.getRelativeDataPath(), detached_path, metadata_snapshot);
}
void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool /* remove_new_dir_if_exists */) const
{
relative_path = new_relative_path;
}
void MergeTreeDataPartInMemory::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const
{
auto it = checksums.files.find("data.bin");
if (it != checksums.files.end())
total_size.data_uncompressed += it->second.uncompressed_size;
for (const auto & column : columns)
each_columns_size[column.name].data_uncompressed += block.getByName(column.name).column->byteSize();
}
IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() const
{
SipHash hash;
IMergeTreeDataPart::Checksum checksum;
for (const auto & column : block)
column.column->updateHashFast(hash);
checksum.uncompressed_size = block.bytes();
hash.get128(checksum.uncompressed_hash.first, checksum.uncompressed_hash.second);
return checksum;
}
DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part)
{
return std::dynamic_pointer_cast<const MergeTreeDataPartInMemory>(part);
}
}

View File

@ -0,0 +1,64 @@
#pragma once
#include <Storages/MergeTree/IMergeTreeDataPart.h>
namespace DB
{
class MergeTreeDataPartInMemory : public IMergeTreeDataPart
{
public:
MergeTreeDataPartInMemory(
const MergeTreeData & storage_,
const String & name_,
const MergeTreePartInfo & info_,
const VolumePtr & volume_,
const std::optional<String> & relative_path_ = {});
MergeTreeDataPartInMemory(
MergeTreeData & storage_,
const String & name_,
const VolumePtr & volume_,
const std::optional<String> & relative_path_ = {});
MergeTreeReaderPtr getReader(
const NamesAndTypesList & columns,
const StorageMetadataPtr & metadata_snapshot,
const MarkRanges & mark_ranges,
UncompressedCache * uncompressed_cache,
MarkCache * mark_cache,
const MergeTreeReaderSettings & reader_settings_,
const ValueSizeMap & avg_value_size_hints,
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override;
MergeTreeWriterPtr getWriter(
const NamesAndTypesList & columns_list,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) const override;
bool isStoredOnDisk() const override { return false; }
bool hasColumnFiles(const String & column_name, const IDataType & /* type */) const override { return !!getColumnPosition(column_name); }
String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; }
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const override;
void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override;
void flushToDisk(const String & base_path, const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const;
/// Returns hash of parts's block
Checksum calculateBlockChecksum() const;
mutable Block block;
private:
mutable std::condition_variable is_merged;
/// Calculates uncompressed sizes in memory.
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
};
using DataPartInMemoryPtr = std::shared_ptr<const MergeTreeDataPartInMemory>;
DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part);
}

View File

@ -229,7 +229,7 @@ String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & colum
return filename;
}
void MergeTreeDataPartWide::calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const
void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const
{
std::unordered_set<String> processed_substreams;
for (const NameAndTypePair & column : columns)

View File

@ -64,7 +64,7 @@ private:
ColumnSize getColumnSizeImpl(const String & name, const IDataType & type, std::unordered_set<String> * processed_substreams) const;
void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
};
}

View File

@ -12,8 +12,9 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_)
: IMergeTreeDataPartWriter(
data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_)
: MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_,
indices_to_recalc_, marks_file_extension_,
default_codec_, settings_, index_granularity_)
{
using DataPart = MergeTreeDataPartCompact;
String data_file_name = DataPart::DATA_FILE_NAME;

View File

@ -1,10 +1,10 @@
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
namespace DB
{
/// Writes data part in compact format.
class MergeTreeDataPartWriterCompact : public IMergeTreeDataPartWriter
class MergeTreeDataPartWriterCompact : public MergeTreeDataPartWriterOnDisk
{
public:
MergeTreeDataPartWriterCompact(

View File

@ -0,0 +1,83 @@
#include <Storages/MergeTree/MergeTreeDataPartWriterInMemory.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergeTreeWriteAheadLog.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
MergeTreeDataPartWriterInMemory::MergeTreeDataPartWriterInMemory(
const DataPartInMemoryPtr & part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const MergeTreeWriterSettings & settings_)
: IMergeTreeDataPartWriter(part_, columns_list_, metadata_snapshot_, settings_)
, part_in_memory(part_) {}
void MergeTreeDataPartWriterInMemory::write(
const Block & block, const IColumn::Permutation * permutation,
const Block & primary_key_block, const Block & /* skip_indexes_block */)
{
if (part_in_memory->block)
throw Exception("DataPartWriterInMemory supports only one write", ErrorCodes::LOGICAL_ERROR);
Block result_block;
if (permutation)
{
for (const auto & col : columns_list)
{
if (primary_key_block.has(col.name))
result_block.insert(primary_key_block.getByName(col.name));
else
{
auto permuted = block.getByName(col.name);
permuted.column = permuted.column->permute(*permutation, 0);
result_block.insert(permuted);
}
}
}
else
{
for (const auto & col : columns_list)
result_block.insert(block.getByName(col.name));
}
index_granularity.appendMark(result_block.rows());
if (with_final_mark)
index_granularity.appendMark(0);
part_in_memory->block = std::move(result_block);
}
void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Block & primary_index_block)
{
size_t rows = primary_index_block.rows();
if (!rows)
return;
size_t primary_columns_num = primary_index_block.columns();
index_columns.resize(primary_columns_num);
for (size_t i = 0; i < primary_columns_num; ++i)
{
const auto & primary_column = *primary_index_block.getByPosition(i).column;
index_columns[i] = primary_column.cloneEmpty();
index_columns[i]->insertFrom(primary_column, 0);
if (with_final_mark)
index_columns[i]->insertFrom(primary_column, rows - 1);
}
}
void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums)
{
/// If part is empty we still need to initialize block by empty columns.
if (!part_in_memory->block)
for (const auto & column : columns_list)
part_in_memory->block.insert(ColumnWithTypeAndName{column.type, column.name});
checksums.files["data.bin"] = part_in_memory->calculateBlockChecksum();
}
}

View File

@ -0,0 +1,29 @@
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
namespace DB
{
/// Writes data part in memory.
class MergeTreeDataPartWriterInMemory : public IMergeTreeDataPartWriter
{
public:
MergeTreeDataPartWriterInMemory(
const DataPartInMemoryPtr & part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot,
const MergeTreeWriterSettings & settings_);
/// You can write only one block. In-memory part can be written only at INSERT.
void write(const Block & block, const IColumn::Permutation * permutation,
const Block & primary_key_block, const Block & skip_indexes_block) override;
void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override;
void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override;
private:
DataPartInMemoryPtr part_in_memory;
};
}

View File

@ -0,0 +1,353 @@
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
#include <utility>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
constexpr auto INDEX_FILE_EXTENSION = ".idx";
}
void MergeTreeDataPartWriterOnDisk::Stream::finalize()
{
compressed.next();
plain_file->next();
marks.next();
}
void MergeTreeDataPartWriterOnDisk::Stream::sync() const
{
plain_file->sync();
marks_file->sync();
}
MergeTreeDataPartWriterOnDisk::Stream::Stream(
const String & escaped_column_name_,
DiskPtr disk_,
const String & data_path_,
const std::string & data_file_extension_,
const std::string & marks_path_,
const std::string & marks_file_extension_,
const CompressionCodecPtr & compression_codec_,
size_t max_compress_block_size_,
size_t estimated_size_,
size_t aio_threshold_) :
escaped_column_name(escaped_column_name_),
data_file_extension{data_file_extension_},
marks_file_extension{marks_file_extension_},
plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite, estimated_size_, aio_threshold_)),
plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_codec_), compressed(compressed_buf),
marks_file(disk_->writeFile(marks_path_ + marks_file_extension, 4096, WriteMode::Rewrite)), marks(*marks_file)
{
}
void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums)
{
String name = escaped_column_name;
checksums.files[name + data_file_extension].is_compressed = true;
checksums.files[name + data_file_extension].uncompressed_size = compressed.count();
checksums.files[name + data_file_extension].uncompressed_hash = compressed.getHash();
checksums.files[name + data_file_extension].file_size = plain_hashing.count();
checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash();
checksums.files[name + marks_file_extension].file_size = marks.count();
checksums.files[name + marks_file_extension].file_hash = marks.getHash();
}
MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list_,
const StorageMetadataPtr & metadata_snapshot_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_)
: IMergeTreeDataPartWriter(data_part_,
columns_list_, metadata_snapshot_, indices_to_recalc_,
index_granularity_, settings_)
, part_path(data_part_->getFullRelativePath())
, marks_file_extension(marks_file_extension_)
, default_codec(default_codec_)
, compute_granularity(index_granularity.empty())
{
if (settings.blocks_are_granules_size && !index_granularity.empty())
throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR);
auto disk = data_part->volume->getDisk();
if (!disk->exists(part_path))
disk->createDirectories(part_path);
}
// Implemetation is splitted into static functions for ability
/// of making unit tests without creation instance of IMergeTreeDataPartWriter,
/// which requires a lot of dependencies and access to filesystem.
static size_t computeIndexGranularityImpl(
const Block & block,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity)
{
size_t rows_in_block = block.rows();
size_t index_granularity_for_block;
if (!can_use_adaptive_index_granularity)
index_granularity_for_block = fixed_index_granularity_rows;
else
{
size_t block_size_in_memory = block.bytes();
if (blocks_are_granules)
index_granularity_for_block = rows_in_block;
else if (block_size_in_memory >= index_granularity_bytes)
{
size_t granules_in_block = block_size_in_memory / index_granularity_bytes;
index_granularity_for_block = rows_in_block / granules_in_block;
}
else
{
size_t size_of_row_in_bytes = block_size_in_memory / rows_in_block;
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes;
}
}
if (index_granularity_for_block == 0) /// very rare case when index granularity bytes less then single row
index_granularity_for_block = 1;
/// We should be less or equal than fixed index granularity
index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block);
return index_granularity_for_block;
}
static void fillIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity,
size_t index_offset,
size_t index_granularity_for_block,
size_t rows_in_block)
{
for (size_t current_row = index_offset; current_row < rows_in_block; current_row += index_granularity_for_block)
index_granularity.appendMark(index_granularity_for_block);
}
size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block)
{
const auto storage_settings = storage.getSettings();
return computeIndexGranularityImpl(
block,
storage_settings->index_granularity_bytes,
storage_settings->index_granularity,
settings.blocks_are_granules_size,
settings.can_use_adaptive_granularity);
}
void MergeTreeDataPartWriterOnDisk::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block)
{
fillIndexGranularityImpl(
index_granularity,
getIndexOffset(),
index_granularity_for_block,
rows_in_block);
}
void MergeTreeDataPartWriterOnDisk::initPrimaryIndex()
{
if (metadata_snapshot->hasPrimaryKey())
{
index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
}
primary_index_initialized = true;
}
void MergeTreeDataPartWriterOnDisk::initSkipIndices()
{
for (const auto & index_helper : skip_indices)
{
String stream_name = index_helper->getFileName();
skip_indices_streams.emplace_back(
std::make_unique<MergeTreeDataPartWriterOnDisk::Stream>(
stream_name,
data_part->volume->getDisk(),
part_path + stream_name, INDEX_FILE_EXTENSION,
part_path + stream_name, marks_file_extension,
default_codec, settings.max_compress_block_size,
0, settings.aio_threshold));
skip_indices_aggregators.push_back(index_helper->createIndexAggregator());
skip_index_filling.push_back(0);
}
skip_indices_initialized = true;
}
void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Block & primary_index_block)
{
if (!primary_index_initialized)
throw Exception("Primary index is not initialized", ErrorCodes::LOGICAL_ERROR);
size_t rows = primary_index_block.rows();
size_t primary_columns_num = primary_index_block.columns();
if (index_columns.empty())
{
index_types = primary_index_block.getDataTypes();
index_columns.resize(primary_columns_num);
last_index_row.resize(primary_columns_num);
for (size_t i = 0; i < primary_columns_num; ++i)
index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty();
}
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
/// Write index. The index contains Primary Key value for each `index_granularity` row.
size_t current_row = getIndexOffset();
size_t total_marks = index_granularity.getMarksCount();
while (index_mark < total_marks && current_row < rows)
{
if (metadata_snapshot->hasPrimaryKey())
{
for (size_t j = 0; j < primary_columns_num; ++j)
{
const auto & primary_column = primary_index_block.getByPosition(j);
index_columns[j]->insertFrom(*primary_column.column, current_row);
primary_column.type->serializeBinary(*primary_column.column, current_row, *index_stream);
}
}
current_row += index_granularity.getMarkRows(index_mark++);
}
/// store last index row to write final mark at the end of column
for (size_t j = 0; j < primary_columns_num; ++j)
{
const IColumn & primary_column = *primary_index_block.getByPosition(j).column.get();
primary_column.get(rows - 1, last_index_row[j]);
}
}
void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block & skip_indexes_block)
{
if (!skip_indices_initialized)
throw Exception("Skip indices are not initialized", ErrorCodes::LOGICAL_ERROR);
size_t rows = skip_indexes_block.rows();
size_t skip_index_current_data_mark = 0;
/// Filling and writing skip indices like in MergeTreeDataPartWriterWide::writeColumn
for (size_t i = 0; i < skip_indices.size(); ++i)
{
const auto index_helper = skip_indices[i];
auto & stream = *skip_indices_streams[i];
size_t prev_pos = 0;
skip_index_current_data_mark = skip_index_data_mark;
while (prev_pos < rows)
{
UInt64 limit = 0;
size_t current_index_offset = getIndexOffset();
if (prev_pos == 0 && current_index_offset != 0)
{
limit = current_index_offset;
}
else
{
limit = index_granularity.getMarkRows(skip_index_current_data_mark);
if (skip_indices_aggregators[i]->empty())
{
skip_indices_aggregators[i] = index_helper->createIndexAggregator();
skip_index_filling[i] = 0;
if (stream.compressed.offset() >= settings.min_compress_block_size)
stream.compressed.next();
writeIntBinary(stream.plain_hashing.count(), stream.marks);
writeIntBinary(stream.compressed.offset(), stream.marks);
/// Actually this numbers is redundant, but we have to store them
/// to be compatible with normal .mrk2 file format
if (settings.can_use_adaptive_granularity)
writeIntBinary(1UL, stream.marks);
}
/// this mark is aggregated, go to the next one
skip_index_current_data_mark++;
}
size_t pos = prev_pos;
skip_indices_aggregators[i]->update(skip_indexes_block, &pos, limit);
if (pos == prev_pos + limit)
{
++skip_index_filling[i];
/// write index if it is filled
if (skip_index_filling[i] == index_helper->index.granularity)
{
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
skip_index_filling[i] = 0;
}
}
prev_pos = pos;
}
}
skip_index_data_mark = skip_index_current_data_mark;
}
void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums)
{
bool write_final_mark = (with_final_mark && data_written);
if (write_final_mark && compute_granularity)
index_granularity.appendMark(0);
if (index_stream)
{
if (write_final_mark)
{
for (size_t j = 0; j < index_columns.size(); ++j)
{
index_columns[j]->insert(last_index_row[j]);
index_types[j]->serializeBinary(last_index_row[j], *index_stream);
}
last_index_row.clear();
}
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
index_stream = nullptr;
}
}
void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(
MergeTreeData::DataPart::Checksums & checksums)
{
for (size_t i = 0; i < skip_indices.size(); ++i)
{
auto & stream = *skip_indices_streams[i];
if (!skip_indices_aggregators[i]->empty())
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
}
for (auto & stream : skip_indices_streams)
{
stream->finalize();
stream->addToChecksums(checksums);
}
skip_indices_streams.clear();
skip_indices_aggregators.clear();
skip_index_filling.clear();
}
}

View File

@ -0,0 +1,130 @@
#pragma once
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromFileBase.h>
#include <Compression/CompressedWriteBuffer.h>
#include <IO/HashingWriteBuffer.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Disks/IDisk.h>
namespace DB
{
/// Writes data part to disk in different formats.
/// Calculates and serializes primary and skip indices if needed.
class MergeTreeDataPartWriterOnDisk : public IMergeTreeDataPartWriter
{
public:
using WrittenOffsetColumns = std::set<std::string>;
/// Helper class, which holds chain of buffers to write data file with marks.
/// It is used to write: one column, skip index or all columns (in compact format).
struct Stream
{
Stream(
const String & escaped_column_name_,
DiskPtr disk_,
const String & data_path_,
const std::string & data_file_extension_,
const std::string & marks_path_,
const std::string & marks_file_extension_,
const CompressionCodecPtr & compression_codec_,
size_t max_compress_block_size_,
size_t estimated_size_,
size_t aio_threshold_);
String escaped_column_name;
std::string data_file_extension;
std::string marks_file_extension;
/// compressed -> compressed_buf -> plain_hashing -> plain_file
std::unique_ptr<WriteBufferFromFileBase> plain_file;
HashingWriteBuffer plain_hashing;
CompressedWriteBuffer compressed_buf;
HashingWriteBuffer compressed;
/// marks -> marks_file
std::unique_ptr<WriteBufferFromFileBase> marks_file;
HashingWriteBuffer marks;
void finalize();
void sync() const;
void addToChecksums(IMergeTreeDataPart::Checksums & checksums);
};
using StreamPtr = std::unique_ptr<Stream>;
MergeTreeDataPartWriterOnDisk(
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns_list,
const StorageMetadataPtr & metadata_snapshot_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const String & marks_file_extension,
const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings,
const MergeTreeIndexGranularity & index_granularity);
void calculateAndSerializePrimaryIndex(const Block & primary_index_block) final;
void calculateAndSerializeSkipIndices(const Block & skip_indexes_block) final;
/// Count index_granularity for block and store in `index_granularity`
size_t computeIndexGranularity(const Block & block);
virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block);
void initSkipIndices() final;
void initPrimaryIndex() final;
void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) final;
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums) final;
void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_)
{
written_offset_columns = written_offset_columns_;
}
protected:
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
using SerializationStates = std::unordered_map<String, SerializationState>;
String part_path;
const String marks_file_extension;
CompressionCodecPtr default_codec;
bool compute_granularity;
bool need_finish_last_granule;
/// Number of marsk in data from which skip indices have to start
/// aggregation. I.e. it's data mark number, not skip indices mark.
size_t skip_index_data_mark = 0;
std::vector<StreamPtr> skip_indices_streams;
MergeTreeIndexAggregators skip_indices_aggregators;
std::vector<size_t> skip_index_filling;
std::unique_ptr<WriteBufferFromFileBase> index_file_stream;
std::unique_ptr<HashingWriteBuffer> index_stream;
DataTypes index_types;
/// Index columns values from the last row from the last block
/// It's written to index file in the `writeSuffixAndFinalizePart` method
Row last_index_row;
bool data_written = false;
bool primary_index_initialized = false;
bool skip_indices_initialized = false;
/// To correctly write Nested elements column-by-column.
WrittenOffsetColumns * written_offset_columns = nullptr;
private:
/// Index is already serialized up to this mark.
size_t index_mark = 0;
};
}

View File

@ -22,8 +22,9 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_)
: IMergeTreeDataPartWriter(
data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_)
: MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_,
indices_to_recalc_, marks_file_extension_,
default_codec_, settings_, index_granularity_)
{
const auto & columns = metadata_snapshot->getColumns();
for (const auto & it : columns_list)

View File

@ -1,10 +1,10 @@
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
namespace DB
{
/// Writes data part in wide format.
class MergeTreeDataPartWriterWide : public IMergeTreeDataPartWriter
class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk
{
public:

View File

@ -250,16 +250,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
new_data_part->minmax_idx = std::move(minmax_idx);
new_data_part->is_temp = true;
/// The name could be non-unique in case of stale files from previous runs.
String full_path = new_data_part->getFullRelativePath();
if (new_data_part->volume->getDisk()->exists(full_path))
if (new_data_part->isStoredOnDisk())
{
LOG_WARNING(log, "Removing old temporary directory {}", fullPath(new_data_part->volume->getDisk(), full_path));
new_data_part->volume->getDisk()->removeRecursive(full_path);
}
/// The name could be non-unique in case of stale files from previous runs.
String full_path = new_data_part->getFullRelativePath();
new_data_part->volume->getDisk()->createDirectories(full_path);
if (new_data_part->volume->getDisk()->exists(full_path))
{
LOG_WARNING(log, "Removing old temporary directory {}", fullPath(new_data_part->volume->getDisk(), full_path));
new_data_part->volume->getDisk()->removeRecursive(full_path);
}
new_data_part->volume->getDisk()->createDirectories(full_path);
}
/// If we need to calculate some columns to sort.
if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices())

View File

@ -17,6 +17,8 @@ struct MergeTreeReaderSettings
struct MergeTreeWriterSettings
{
MergeTreeWriterSettings() = default;
MergeTreeWriterSettings(const Settings & global_settings, bool can_use_adaptive_granularity_,
size_t aio_threshold_, bool blocks_are_granules_size_ = false)
: min_compress_block_size(global_settings.min_compress_block_size)
@ -31,6 +33,7 @@ struct MergeTreeWriterSettings
bool can_use_adaptive_granularity;
bool blocks_are_granules_size;
/// true if we write temporary files during alter.
size_t estimated_size = 0;
};

View File

@ -72,6 +72,8 @@ size_t MergeTreeIndexGranularityInfo::getMarkSizeInBytes(size_t columns_num) con
return is_adaptive ? getAdaptiveMrkSizeWide() : getNonAdaptiveMrkSizeWide();
else if (type == MergeTreeDataPartType::COMPACT)
return getAdaptiveMrkSizeCompact(columns_num);
else if (type == MergeTreeDataPartType::IN_MEMORY)
return 0;
else
throw Exception("Unknown part type", ErrorCodes::UNKNOWN_PART_TYPE);
}
@ -88,6 +90,8 @@ std::string getAdaptiveMrkExtension(MergeTreeDataPartType part_type)
return ".mrk2";
else if (part_type == MergeTreeDataPartType::COMPACT)
return ".mrk3";
else if (part_type == MergeTreeDataPartType::IN_MEMORY)
return "";
else
throw Exception("Unknown part type", ErrorCodes::UNKNOWN_PART_TYPE);
}

View File

@ -158,4 +158,22 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr
checksums.files["partition.dat"].file_hash = out_hashing.getHash();
}
void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row)
{
if (!metadata_snapshot->hasPartitionKey())
return;
const auto & partition_key = metadata_snapshot->getPartitionKey();
partition_key.expression->execute(block);
size_t partition_columns_num = partition_key.sample_block.columns();
value.resize(partition_columns_num);
for (size_t i = 0; i < partition_columns_num; ++i)
{
const auto & column_name = partition_key.sample_block.getByPosition(i).name;
const auto & partition_column = block.getByName(column_name).column;
partition_column->get(row, value[i]);
}
}
}

View File

@ -12,6 +12,9 @@ class Block;
class MergeTreeData;
struct FormatSettings;
struct MergeTreeDataPartChecksums;
struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
/// This class represents a partition value of a single part and encapsulates its loading/storing logic.
struct MergeTreePartition
@ -36,6 +39,8 @@ public:
void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
void assign(const MergeTreePartition & other) { value.assign(other.value); }
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row);
};
}

View File

@ -94,12 +94,11 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
if (!position && typeid_cast<const DataTypeArray *>(type.get()))
{
/// If array of Nested column is missing in part,
/// we have to read it's offsets if they exists.
/// we have to read its offsets if they exist.
position = findColumnForOffsets(name);
read_only_offsets[i] = (position != std::nullopt);
}
column_positions[i] = std::move(position);
}
@ -112,6 +111,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading,
size_t read_rows = 0;
size_t num_columns = columns.size();
checkNumberOfColumns(num_columns);
MutableColumns mutable_columns(num_columns);
auto column_it = columns.begin();
@ -177,23 +177,6 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading,
return read_rows;
}
MergeTreeReaderCompact::ColumnPosition MergeTreeReaderCompact::findColumnForOffsets(const String & column_name)
{
String table_name = Nested::extractTableName(column_name);
for (const auto & part_column : data_part->getColumns())
{
if (typeid_cast<const DataTypeArray *>(part_column.type.get()))
{
auto position = data_part->getColumnPosition(part_column.name);
if (position && Nested::extractTableName(part_column.name) == table_name)
return position;
}
}
return {};
}
void MergeTreeReaderCompact::readData(
const String & name, IColumn & column, const IDataType & type,
size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets)

View File

@ -41,7 +41,6 @@ private:
MergeTreeMarksLoader marks_loader;
using ColumnPosition = std::optional<size_t>;
/// Positions of columns in part structure.
std::vector<ColumnPosition> column_positions;
/// Should we read full column or only it's offsets
@ -54,8 +53,6 @@ private:
void readData(const String & name, IColumn & column, const IDataType & type,
size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false);
ColumnPosition findColumnForOffsets(const String & column_name);
};
}

View File

@ -0,0 +1,105 @@
#include <Storages/MergeTree/MergeTreeReaderInMemory.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h>
#include <Columns/ColumnArray.h>
#include <Poco/File.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_ALL_DATA;
extern const int ARGUMENT_OUT_OF_BOUND;
}
MergeTreeReaderInMemory::MergeTreeReaderInMemory(
DataPartInMemoryPtr data_part_,
NamesAndTypesList columns_,
const StorageMetadataPtr & metadata_snapshot_,
MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_)
: IMergeTreeReader(data_part_, std::move(columns_), metadata_snapshot_,
nullptr, nullptr, std::move(mark_ranges_),
std::move(settings_), {})
, part_in_memory(std::move(data_part_))
{
for (const auto & name_and_type : columns)
{
auto [name, type] = getColumnFromPart(name_and_type);
/// If array of Nested column is missing in part,
/// we have to read its offsets if they exist.
if (!part_in_memory->block.has(name) && typeid_cast<const DataTypeArray *>(type.get()))
if (auto offset_position = findColumnForOffsets(name))
positions_for_offsets[name] = *offset_position;
}
}
size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
{
if (!continue_reading)
total_rows_read = 0;
size_t total_marks = data_part->index_granularity.getMarksCount();
if (from_mark >= total_marks)
throw Exception("Mark " + toString(from_mark) + " is out of bound. Max mark: "
+ toString(total_marks), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
size_t num_columns = res_columns.size();
checkNumberOfColumns(num_columns);
size_t part_rows = part_in_memory->block.rows();
if (total_rows_read >= part_rows)
throw Exception("Cannot read data in MergeTreeReaderInMemory. Rows already read: "
+ toString(total_rows_read) + ". Rows in part: " + toString(part_rows), ErrorCodes::CANNOT_READ_ALL_DATA);
size_t rows_to_read = std::min(max_rows_to_read, part_rows - total_rows_read);
auto column_it = columns.begin();
for (size_t i = 0; i < num_columns; ++i, ++column_it)
{
auto [name, type] = getColumnFromPart(*column_it);
/// Copy offsets, if array of Nested column is missing in part.
auto offsets_it = positions_for_offsets.find(name);
if (offsets_it != positions_for_offsets.end())
{
const auto & source_offsets = assert_cast<const ColumnArray &>(
*part_in_memory->block.getByPosition(offsets_it->second).column).getOffsets();
if (res_columns[i] == nullptr)
res_columns[i] = type->createColumn();
auto mutable_column = res_columns[i]->assumeMutable();
auto & res_offstes = assert_cast<ColumnArray &>(*mutable_column).getOffsets();
for (size_t row = 0; row < rows_to_read; ++row)
res_offstes.push_back(source_offsets[total_rows_read + row]);
res_columns[i] = std::move(mutable_column);
}
else if (part_in_memory->block.has(name))
{
const auto & block_column = part_in_memory->block.getByName(name).column;
if (rows_to_read == part_rows)
{
res_columns[i] = block_column;
}
else
{
if (res_columns[i] == nullptr)
res_columns[i] = type->createColumn();
auto mutable_column = res_columns[i]->assumeMutable();
mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_to_read);
res_columns[i] = std::move(mutable_column);
}
}
}
total_rows_read += rows_to_read;
return rows_to_read;
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Storages/MergeTree/IMergeTreeReader.h>
namespace DB
{
class MergeTreeDataPartInMemory;
using DataPartInMemoryPtr = std::shared_ptr<const MergeTreeDataPartInMemory>;
/// Reader for InMemory parts
class MergeTreeReaderInMemory : public IMergeTreeReader
{
public:
MergeTreeReaderInMemory(
DataPartInMemoryPtr data_part_,
NamesAndTypesList columns_,
const StorageMetadataPtr & metadata_snapshot_,
MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_);
/// Return the number of rows has been read or zero if there is no columns to read.
/// If continue_reading is true, continue reading from last state, otherwise seek to from_mark
size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override;
bool canReadIncompleteGranules() const override { return true; }
private:
size_t total_rows_read = 0;
DataPartInMemoryPtr part_in_memory;
std::unordered_map<String, size_t> positions_for_offsets;
};
}

View File

@ -21,7 +21,6 @@ namespace
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int MEMORY_LIMIT_EXCEEDED;
}
@ -68,11 +67,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si
try
{
size_t num_columns = columns.size();
if (res_columns.size() != num_columns)
throw Exception("invalid number of columns passed to MergeTreeReader::readRows. "
"Expected " + toString(num_columns) + ", "
"got " + toString(res_columns.size()), ErrorCodes::LOGICAL_ERROR);
checkNumberOfColumns(num_columns);
/// Pointers to offset columns that are common to the nested data structure columns.
/// If append is true, then the value will be equal to nullptr and will be used only to

View File

@ -38,8 +38,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
data_part{owned_data_part_},
all_mark_ranges(std::move(mark_ranges_)),
part_index_in_query(part_index_in_query_),
check_columns(check_columns_),
path(data_part->getFullRelativePath())
check_columns(check_columns_)
{
/// Let's estimate total number of rows for progress bar.
for (const auto & range : all_mark_ranges)

View File

@ -65,7 +65,6 @@ private:
size_t part_index_in_query = 0;
bool check_columns;
String path;
bool is_first_task = true;
Poco::Logger * log = &Poco::Logger::get("MergeTreeSelectProcessor");

View File

@ -31,6 +31,10 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
/** Data storing format settings. */ \
M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \
M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \
M(SettingUInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \
M(SettingUInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \
M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \
M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \
\
/** Merge settings. */ \
M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \
@ -119,7 +123,8 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
static bool isPartFormatSetting(const String & name)
{
return name == "min_bytes_for_wide_part" || name == "min_rows_for_wide_part";
return name == "min_bytes_for_wide_part" || name == "min_rows_for_wide_part"
|| name == "min_bytes_for_compact_part" || name == "min_rows_for_compact_part";
}
};

View File

@ -61,7 +61,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask()
return false;
}
const std::string path = task->data_part->getFullRelativePath();
const std::string part_name = task->data_part->name;
/// Allows pool to reduce number of threads in case of too slow reads.
auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); };
@ -86,7 +86,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask()
else
{
/// in other case we can reuse readers, anyway they will be "seeked" to required mark
if (path != last_readed_part_path)
if (part_name != last_readed_part_name)
{
auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]);
/// retain avg_value_size_hints
@ -101,7 +101,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask()
}
}
last_readed_part_path = path;
last_readed_part_name = part_name;
return true;
}

View File

@ -44,7 +44,7 @@ private:
size_t min_marks_to_read;
/// Last part readed in this thread
std::string last_readed_part_path;
std::string last_readed_part_name;
/// Names from header. Used in order to order columns in read blocks.
Names ordered_names;
};

View File

@ -0,0 +1,196 @@
#include <Storages/MergeTree/MergeTreeWriteAheadLog.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <IO/ReadHelpers.h>
#include <Poco/File.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FORMAT_VERSION;
extern const int CANNOT_READ_ALL_DATA;
extern const int BAD_DATA_PART_NAME;
extern const int CORRUPTED_DATA;
}
MergeTreeWriteAheadLog::MergeTreeWriteAheadLog(
const MergeTreeData & storage_,
const DiskPtr & disk_,
const String & name_)
: storage(storage_)
, disk(disk_)
, name(name_)
, path(storage.getRelativeDataPath() + name_)
{
init();
}
void MergeTreeWriteAheadLog::init()
{
out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append);
/// Small hack: in NativeBlockOutputStream header is used only in `getHeader` method.
/// To avoid complex logic of changing it during ALTERs we leave it empty.
block_out = std::make_unique<NativeBlockOutputStream>(*out, 0, Block{});
min_block_number = std::numeric_limits<Int64>::max();
max_block_number = -1;
}
void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name)
{
std::lock_guard lock(write_mutex);
auto part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version);
min_block_number = std::min(min_block_number, part_info.min_block);
max_block_number = std::max(max_block_number, part_info.max_block);
writeIntBinary(static_cast<UInt8>(0), *out); /// version
writeIntBinary(static_cast<UInt8>(ActionType::ADD_PART), *out);
writeStringBinary(part_name, *out);
block_out->write(block);
block_out->flush();
auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes;
if (out->count() > max_wal_bytes)
rotate(lock);
}
void MergeTreeWriteAheadLog::dropPart(const String & part_name)
{
std::lock_guard lock(write_mutex);
writeIntBinary(static_cast<UInt8>(0), *out);
writeIntBinary(static_cast<UInt8>(ActionType::DROP_PART), *out);
writeStringBinary(part_name, *out);
}
void MergeTreeWriteAheadLog::rotate(const std::lock_guard<std::mutex> &)
{
String new_name = String(WAL_FILE_NAME) + "_"
+ toString(min_block_number) + "_"
+ toString(max_block_number) + WAL_FILE_EXTENSION;
disk->replaceFile(path, storage.getRelativeDataPath() + new_name);
init();
}
MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot)
{
std::lock_guard lock(write_mutex);
MergeTreeData::MutableDataPartsVector parts;
auto in = disk->readFile(path, DBMS_DEFAULT_BUFFER_SIZE);
NativeBlockInputStream block_in(*in, 0);
NameSet dropped_parts;
while (!in->eof())
{
MergeTreeData::MutableDataPartPtr part;
UInt8 version;
String part_name;
Block block;
ActionType action_type;
try
{
readIntBinary(version, *in);
if (version != 0)
throw Exception("Unknown WAL format version: " + toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION);
readIntBinary(action_type, *in);
readStringBinary(part_name, *in);
if (action_type == ActionType::DROP_PART)
{
dropped_parts.insert(part_name);
}
else if (action_type == ActionType::ADD_PART)
{
auto part_disk = storage.reserveSpace(0)->getDisk();
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk);
part = storage.createPart(
part_name,
MergeTreeDataPartType::IN_MEMORY,
MergeTreePartInfo::fromPartName(part_name, storage.format_version),
single_disk_volume,
part_name);
block = block_in.read();
}
else
{
throw Exception("Unknown action type: " + toString(static_cast<UInt8>(action_type)), ErrorCodes::CORRUPTED_DATA);
}
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::CANNOT_READ_ALL_DATA
|| e.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION
|| e.code() == ErrorCodes::BAD_DATA_PART_NAME
|| e.code() == ErrorCodes::CORRUPTED_DATA)
{
LOG_WARNING(&Poco::Logger::get(storage.getLogName() + " (WriteAheadLog)"),
"WAL file '{}' is broken. {}", path, e.displayText());
/// If file is broken, do not write new parts to it.
/// But if it contains any part rotate and save them.
if (max_block_number == -1)
disk->remove(path);
else if (name == DEFAULT_WAL_FILE_NAME)
rotate(lock);
break;
}
throw;
}
if (action_type == ActionType::ADD_PART)
{
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, nullptr);
part->minmax_idx.update(block, storage.minmax_idx_columns);
part->partition.create(metadata_snapshot, block, 0);
if (metadata_snapshot->hasSortingKey())
metadata_snapshot->getSortingKey().expression->execute(block);
part_out.writePrefix();
part_out.write(block);
part_out.writeSuffixAndFinalizePart(part);
min_block_number = std::min(min_block_number, part->info.min_block);
max_block_number = std::max(max_block_number, part->info.max_block);
parts.push_back(std::move(part));
}
}
MergeTreeData::MutableDataPartsVector result;
std::copy_if(parts.begin(), parts.end(), std::back_inserter(result),
[&dropped_parts](const auto & part) { return dropped_parts.count(part->name) == 0; });
return result;
}
std::optional<MergeTreeWriteAheadLog::MinMaxBlockNumber>
MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(const String & filename)
{
Int64 min_block;
Int64 max_block;
ReadBufferFromString in(filename);
if (!checkString(WAL_FILE_NAME, in)
|| !checkChar('_', in)
|| !tryReadIntText(min_block, in)
|| !checkChar('_', in)
|| !tryReadIntText(max_block, in))
{
return {};
}
return std::make_pair(min_block, max_block);
}
}

View File

@ -0,0 +1,62 @@
#pragma once
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Disks/IDisk.h>
namespace DB
{
class MergeTreeData;
/** WAL stores addditions and removals of data parts in in-memory format.
* Format of data in WAL:
* - version
* - type of action (ADD or DROP)
* - part name
* - part's block in Native format. (for ADD action)
*/
class MergeTreeWriteAheadLog
{
public:
/// Append-only enum. It is serialized to WAL
enum class ActionType : UInt8
{
ADD_PART = 0,
DROP_PART = 1,
};
constexpr static auto WAL_FILE_NAME = "wal";
constexpr static auto WAL_FILE_EXTENSION = ".bin";
constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin";
MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_,
const String & name = DEFAULT_WAL_FILE_NAME);
void addPart(const Block & block, const String & part_name);
void dropPart(const String & part_name);
std::vector<MergeTreeMutableDataPartPtr> restore(const StorageMetadataPtr & metadata_snapshot);
using MinMaxBlockNumber = std::pair<Int64, Int64>;
static std::optional<MinMaxBlockNumber> tryParseMinMaxBlockNumber(const String & filename);
private:
void init();
void rotate(const std::lock_guard<std::mutex> & lock);
const MergeTreeData & storage;
DiskPtr disk;
String name;
String path;
std::unique_ptr<WriteBuffer> out;
std::unique_ptr<NativeBlockOutputStream> block_out;
Int64 min_block_number = std::numeric_limits<Int64>::max();
Int64 max_block_number = -1;
mutable std::mutex write_mutex;
};
}

View File

@ -44,8 +44,11 @@ MergedBlockOutputStream::MergedBlockOutputStream(
: IMergedBlockOutputStream(data_part, metadata_snapshot_)
, columns_list(columns_list_)
{
MergeTreeWriterSettings writer_settings(data_part->storage.global_context.getSettings(),
data_part->storage.canUseAdaptiveGranularity(), aio_threshold, blocks_are_granules_size);
MergeTreeWriterSettings writer_settings(
storage.global_context.getSettings(),
storage.canUseAdaptiveGranularity(),
aio_threshold,
blocks_are_granules_size);
if (aio_threshold > 0 && !merged_column_to_size.empty())
{
@ -57,7 +60,8 @@ MergedBlockOutputStream::MergedBlockOutputStream(
}
}
volume->getDisk()->createDirectories(part_path);
if (!part_path.empty())
volume->getDisk()->createDirectories(part_path);
writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings);
writer->initPrimaryIndex();
@ -105,6 +109,24 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
else
part_columns = *total_columns_list;
if (new_part->isStoredOnDisk())
finalizePartOnDisk(new_part, part_columns, checksums);
new_part->setColumns(part_columns);
new_part->rows_count = rows_count;
new_part->modification_time = time(nullptr);
new_part->index = writer->releaseIndexColumns();
new_part->checksums = checksums;
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->index_granularity = writer->getIndexGranularity();
new_part->calculateColumnsSizesOnDisk();
}
void MergedBlockOutputStream::finalizePartOnDisk(
const MergeTreeData::MutableDataPartPtr & new_part,
NamesAndTypesList & part_columns,
MergeTreeData::DataPart::Checksums & checksums)
{
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part))
{
new_part->partition.store(storage, volume->getDisk(), part_path, checksums);
@ -145,15 +167,6 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096);
checksums.write(*out);
}
new_part->setColumns(part_columns);
new_part->rows_count = rows_count;
new_part->modification_time = time(nullptr);
new_part->index = writer->releaseIndexColumns();
new_part->checksums = checksums;
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->index_granularity = writer->getIndexGranularity();
new_part->calculateColumnsSizesOnDisk();
}
void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Permutation * permutation)
@ -173,8 +186,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, permutation);
writer->write(block, permutation, primary_key_block, skip_indexes_block);
writer->calculateAndSerializeSkipIndices(skip_indexes_block, rows);
writer->calculateAndSerializePrimaryIndex(primary_key_block, rows);
writer->calculateAndSerializeSkipIndices(skip_indexes_block);
writer->calculateAndSerializePrimaryIndex(primary_key_block);
writer->next();
rows_count += rows;

View File

@ -55,9 +55,14 @@ private:
*/
void writeImpl(const Block & block, const IColumn::Permutation * permutation);
void finalizePartOnDisk(
const MergeTreeData::MutableDataPartPtr & new_part,
NamesAndTypesList & part_columns,
MergeTreeData::DataPart::Checksums & checksums);
private:
NamesAndTypesList columns_list;
IMergeTreeDataPart::MinMaxIndex minmax_idx;
size_t rows_count = 0;
};

View File

@ -1,4 +1,5 @@
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
#include <Interpreters/Context.h>
namespace DB
@ -34,8 +35,12 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
std::move(writer_settings),
index_granularity);
writer->setWrittenOffsetColumns(offset_columns_);
writer->initSkipIndices();
auto * writer_on_disk = dynamic_cast<MergeTreeDataPartWriterOnDisk *>(writer.get());
if (!writer_on_disk)
throw Exception("MergedColumnOnlyOutputStream supports only parts stored on disk", ErrorCodes::NOT_IMPLEMENTED);
writer_on_disk->setWrittenOffsetColumns(offset_columns_);
writer_on_disk->initSkipIndices();
}
void MergedColumnOnlyOutputStream::write(const Block & block)
@ -48,12 +53,11 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, nullptr);
size_t rows = block.rows();
if (!rows)
if (!block.rows())
return;
writer->write(block);
writer->calculateAndSerializeSkipIndices(skip_indexes_block, rows);
writer->calculateAndSerializeSkipIndices(skip_indexes_block);
writer->next();
}

View File

@ -250,6 +250,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(
log_entry.new_part_name = part_name;
log_entry.quorum = quorum;
log_entry.block_id = block_id;
log_entry.new_part_type = part->getType();
/// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock.

View File

@ -56,6 +56,7 @@ void ReplicatedMergeTreeCleanupThread::run()
void ReplicatedMergeTreeCleanupThread::iterate()
{
storage.clearOldPartsAndRemoveFromZK();
storage.clearOldWriteAheadLogs();
{
auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations);

View File

@ -1025,7 +1025,12 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
if (part)
sum_parts_size_in_bytes += part->getBytesOnDisk();
{
if (auto part_in_memory = asInMemoryPart(part))
sum_parts_size_in_bytes += part_in_memory->block.bytes();
else
sum_parts_size_in_bytes += part->getBytesOnDisk();
}
}
if (merger_mutator.merges_blocker.isCancelled())

View File

@ -7,6 +7,7 @@
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Compression/CompressedReadBuffer.h>
#include <IO/HashingReadBuffer.h>
#include <Common/CurrentMetrics.h>
@ -161,11 +162,22 @@ IMergeTreeDataPart::Checksums checkDataPart(
return checksums_data;
}
IMergeTreeDataPart::Checksums checkDataPartInMemory(const DataPartInMemoryPtr & data_part)
{
IMergeTreeDataPart::Checksums data_checksums;
data_checksums.files["data.bin"] = data_part->calculateBlockChecksum();
data_part->checksums.checkEqual(data_checksums, true);
return data_checksums;
}
IMergeTreeDataPart::Checksums checkDataPart(
MergeTreeData::DataPartPtr data_part,
bool require_checksums,
std::function<bool()> is_cancelled)
{
if (auto part_in_memory = asInMemoryPart(data_part))
return checkDataPartInMemory(part_in_memory);
return checkDataPart(
data_part->volume->getDisk(),
data_part->getFullRelativePath(),

View File

@ -20,6 +20,7 @@
#include <Storages/AlterCommands.h>
#include <Storages/PartitionCommands.h>
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Disks/StoragePolicy.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/checkDataPart.h>
@ -87,6 +88,7 @@ StorageMergeTree::StorageMergeTree(
void StorageMergeTree::startup()
{
clearOldPartsFromFilesystem();
clearOldWriteAheadLogs();
/// Temporary directories contain incomplete results of merges (after forced restart)
/// and don't allow to reinitialize them, so delete each of them immediately
@ -195,7 +197,10 @@ std::optional<UInt64> StorageMergeTree::totalBytes() const
BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
{
return std::make_shared<MergeTreeBlockOutputStream>(*this, metadata_snapshot, context.getSettingsRef().max_partitions_per_insert_block);
const auto & settings = context.getSettingsRef();
return std::make_shared<MergeTreeBlockOutputStream>(
*this, metadata_snapshot, settings.max_partitions_per_insert_block);
}
void StorageMergeTree::checkTableCanBeDropped() const
@ -695,8 +700,8 @@ bool StorageMergeTree::merge(
new_part = merger_mutator.mergePartsToTemporaryPart(
future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr),
merging_tagger->reserved_space, deduplicate, force_ttl);
merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr);
merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr);
merging_tagger->is_successful = true;
write_part_log({});
}
@ -878,13 +883,13 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask()
clearOldTemporaryDirectories();
}
clearOldMutations();
clearOldWriteAheadLogs();
}
///TODO: read deduplicate option from table config
if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/))
return BackgroundProcessingPoolTaskResult::SUCCESS;
if (tryMutatePart())
return BackgroundProcessingPoolTaskResult::SUCCESS;
@ -1073,14 +1078,14 @@ void StorageMergeTree::alterPartition(
case PartitionCommand::FREEZE_PARTITION:
{
auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
freezePartition(command.partition, command.with_name, context, lock);
freezePartition(command.partition, metadata_snapshot, command.with_name, context, lock);
}
break;
case PartitionCommand::FREEZE_ALL_PARTITIONS:
{
auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
freezeAll(command.with_name, context, lock);
freezeAll(command.with_name, metadata_snapshot, context, lock);
}
break;
@ -1097,6 +1102,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = merger_mutator.merges_blocker.cancel();
auto metadata_snapshot = getInMemoryMetadataPtr();
String partition_id = getPartitionIDFromQuery(partition, context);
/// TODO: should we include PreComitted parts like in Replicated case?
@ -1110,7 +1116,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons
for (const auto & part : parts_to_remove)
{
LOG_INFO(log, "Detaching {}", part->relative_path);
part->makeCloneInDetached("");
part->makeCloneInDetached("", metadata_snapshot);
}
}
@ -1171,7 +1177,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
dst_parts.emplace_back(cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info));
dst_parts.emplace_back(cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot));
}
/// ATTACH empty part set
@ -1255,7 +1261,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info));
dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot));
}
/// empty part set
@ -1324,7 +1330,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & c
/// If the checksums file is not present, calculate the checksums and write them to disk.
String checksums_path = part_path + "checksums.txt";
String tmp_checksums_path = part_path + "checksums.txt.tmp";
if (!disk->exists(checksums_path))
if (part->isStoredOnDisk() && !disk->exists(checksums_path))
{
try
{

View File

@ -314,6 +314,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
/// Temporary directories contain untinalized results of Merges or Fetches (after forced restart)
/// and don't allow to reinitialize them, so delete each of them immediately
clearOldTemporaryDirectories(0);
clearOldWriteAheadLogs();
}
createNewZooKeeperNodes();
@ -1524,6 +1525,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
/// Looking for covering part. After that entry.actual_new_part_name may be filled.
String replica = findReplicaHavingCoveringPart(entry, true);
const auto storage_settings_ptr = getSettings();
auto metadata_snapshot = getInMemoryMetadataPtr();
static std::atomic_uint total_fetches {0};
if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches)
@ -1656,7 +1658,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
try
{
String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name;
if (!fetchPart(part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
if (!fetchPart(part_name, metadata_snapshot, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
return false;
}
catch (Exception & e)
@ -1720,6 +1722,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
/// And, if you do not, the parts will come to life after the server is restarted.
/// Therefore, we use all data parts.
auto metadata_snapshot = getInMemoryMetadataPtr();
DataPartsVector parts_to_remove;
{
auto data_parts_lock = lockParts();
@ -1732,7 +1735,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
for (const auto & part : parts_to_remove)
{
LOG_INFO(log, "Detaching {}", part->relative_path);
part->makeCloneInDetached("");
part->makeCloneInDetached("", metadata_snapshot);
}
}
@ -1755,6 +1758,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
{
Stopwatch watch;
auto & entry_replace = *entry.replace_range_entry;
auto metadata_snapshot = getInMemoryMetadataPtr();
MergeTreePartInfo drop_range = MergeTreePartInfo::fromPartName(entry_replace.drop_range_part_name, format_version);
/// Range with only one block has special meaning ATTACH PARTITION
@ -1994,7 +1998,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
throw Exception("Checksums of " + part_desc->src_table_part->name + " is suddenly changed", ErrorCodes::UNFINISHED);
part_desc->res_part = cloneAndLoadDataPartOnSameDisk(
part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info);
part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot);
}
else if (!part_desc->replica.empty())
{
@ -2007,7 +2011,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
if (interserver_scheme != address.scheme)
throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR);
part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, source_replica_path,
part_desc->res_part = fetcher.fetchPart(
metadata_snapshot, part_desc->found_new_part_name, source_replica_path,
address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_");
/// TODO: check columns_version of fetched part
@ -3067,7 +3072,8 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id)
}
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & source_replica_path, bool to_detached, size_t quorum)
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot,
const String & source_replica_path, bool to_detached, size_t quorum)
{
const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version);
@ -3153,7 +3159,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
{
get_part = [&, part_to_clone]()
{
return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info);
return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot);
};
}
else
@ -3171,7 +3177,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
ErrorCodes::LOGICAL_ERROR);
return fetcher.fetchPart(
part_name, source_replica_path,
metadata_snapshot, part_name, source_replica_path,
address.host, address.replication_port,
timeouts, user_password.first, user_password.second, interserver_scheme, to_detached);
};
@ -3211,7 +3217,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
}
else
{
part->renameTo("detached/" + part_name);
part->renameTo("detached/" + part_name, true);
}
}
catch (...)
@ -3468,7 +3474,11 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/,
const Settings & query_settings = context.getSettingsRef();
bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate;
return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this, metadata_snapshot, query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate);
return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(
*this, metadata_snapshot, query_settings.insert_quorum,
query_settings.insert_quorum_timeout.totalMilliseconds(),
query_settings.max_partitions_per_insert_block,
deduplicate);
}
@ -3892,20 +3902,20 @@ void StorageReplicatedMergeTree::alterPartition(
break;
case PartitionCommand::FETCH_PARTITION:
fetchPartition(command.partition, command.from_zookeeper_path, query_context);
fetchPartition(command.partition, metadata_snapshot, command.from_zookeeper_path, query_context);
break;
case PartitionCommand::FREEZE_PARTITION:
{
auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout);
freezePartition(command.partition, command.with_name, query_context, lock);
freezePartition(command.partition, metadata_snapshot, command.with_name, query_context, lock);
}
break;
case PartitionCommand::FREEZE_ALL_PARTITIONS:
{
auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout);
freezeAll(command.with_name, query_context, lock);
freezeAll(command.with_name, metadata_snapshot, query_context, lock);
}
break;
}
@ -4501,7 +4511,7 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t
}
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & query_context)
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from_, const Context & query_context)
{
String partition_id = getPartitionIDFromQuery(partition, query_context);
@ -4641,7 +4651,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
{
try
{
fetchPart(part, best_replica_path, true, 0);
fetchPart(part, metadata_snapshot, best_replica_path, true, 0);
}
catch (const DB::Exception & e)
{
@ -5103,8 +5113,8 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", to_delete_futures.size(), partition_id);
}
void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace,
const Context & context)
void StorageReplicatedMergeTree::replacePartitionFrom(
const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context)
{
/// First argument is true, because we possibly will add new data to current table.
auto lock1 = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
@ -5184,7 +5194,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
UInt64 index = lock->getNumber();
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info);
auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot);
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);
@ -5363,7 +5373,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
UInt64 index = lock->getNumber();
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info);
auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot);
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);

View File

@ -480,7 +480,7 @@ private:
* If quorum != 0, then the node for tracking the quorum is updated.
* Returns false if part is already fetching right now.
*/
bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum);
bool fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & replica_path, bool to_detached, size_t quorum);
/// Required only to avoid races between executeLogEntry and fetchPartition
std::unordered_set<String> currently_fetching_parts;
@ -539,7 +539,7 @@ private:
void attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context);
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context);
void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context);
void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context);
void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context);
/// Check granularity of already existing replicated table in zookeeper if it exists
/// return true if it's fixed

View File

@ -119,8 +119,16 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto
columns_[i++]->insert(info.database);
columns_[i++]->insert(info.table);
columns_[i++]->insert(info.engine);
columns_[i++]->insert(part->volume->getDisk()->getName());
columns_[i++]->insert(part->getFullPath());
if (part->isStoredOnDisk())
{
columns_[i++]->insert(part->volume->getDisk()->getName());
columns_[i++]->insert(part->getFullPath());
}
else
{
columns_[i++]->insertDefault();
columns_[i++]->insertDefault();
}
if (has_state_column)
columns_[i++]->insert(part->stateString());

View File

@ -3,7 +3,7 @@
#include <Columns/ColumnVector.h>
// I know that inclusion of .cpp is not good at all
#include <Storages/MergeTree/IMergeTreeDataPartWriter.cpp>
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp>
using namespace DB;
static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes)

View File

@ -82,6 +82,11 @@ SRCS(
MergeTree/ReplicatedMergeTreeTableMetadata.cpp
MergeTree/SimpleMergeSelector.cpp
MergeTree/TTLMergeSelector.cpp
MergeTree/MergeTreeDataPartInMemory.cpp
MergeTree/MergeTreeDataPartWriterInMemory.cpp
MergeTree/MergeTreeDataPartWriterOnDisk.cpp
MergeTree/MergeTreeReaderInMemory.cpp
MergeTree/MergeTreeWriteAheadLog.cpp
System/attachSystemTables.cpp
System/StorageSystemAggregateFunctionCombinators.cpp
System/StorageSystemAsynchronousMetrics.cpp

View File

@ -13,7 +13,7 @@ class PartitionManager:
Can act as a context manager:
with pm as PartitionManager():
with PartitionManager() as pm:
pm.partition_instances(instance1, instance2)
...
# At exit all partitions are removed automatically.

View File

@ -0,0 +1,8 @@
<yandex>
<merge_tree>
<max_bytes_to_merge_at_min_space_in_pool>1</max_bytes_to_merge_at_min_space_in_pool>
<max_bytes_to_merge_at_max_space_in_pool>2</max_bytes_to_merge_at_max_space_in_pool>
<number_of_free_entries_in_pool_to_lower_max_size_of_merge>100</number_of_free_entries_in_pool_to_lower_max_size_of_merge>
<max_replicated_merges_in_queue>0</max_replicated_merges_in_queue>
</merge_tree>
</yandex>

View File

@ -8,6 +8,8 @@ import struct
from helpers.test_tools import TSV
from helpers.test_tools import assert_eq_with_retry
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from multiprocessing.dummy import Pool
cluster = ClickHouseCluster(__file__)
@ -38,8 +40,9 @@ def create_tables(name, nodes, node_settings, shard):
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes},
min_rows_for_wide_part = {min_rows_for_wide_part}, min_bytes_for_wide_part = {min_bytes_for_wide_part}
SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes},
min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part},
in_memory_parts_enable_wal = 1
'''.format(name=name, shard=shard, repl=i, **settings))
def create_tables_old_format(name, nodes, shard):
@ -53,28 +56,44 @@ def create_tables_old_format(name, nodes, shard):
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True)
settings_default = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0}
settings_not_adaptive = {'index_granularity' : 64, 'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0}
settings_default = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
settings_compact_only = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_rows_for_compact_part' : 0}
settings_not_adaptive = {'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True)
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/no_leader.xml'], with_zookeeper=True)
settings_compact = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0}
settings_wide = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_bytes_for_wide_part' : 0}
settings_compact = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
settings_wide = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_rows_for_compact_part' : 0}
node5 = cluster.add_instance('node5', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256}
node9 = cluster.add_instance('node9', config_dir="configs", with_zookeeper=True, stay_alive=True)
node10 = cluster.add_instance('node10', config_dir="configs", with_zookeeper=True)
node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True)
node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
create_tables('polymorphic_table', [node1, node2], [settings_default, settings_default], "shard1")
create_tables('compact_parts_only', [node1, node2], [settings_compact_only, settings_compact_only], "shard1")
create_tables('non_adaptive_table', [node1, node2], [settings_not_adaptive, settings_default], "shard1")
create_tables('polymorphic_table_compact', [node3, node4], [settings_compact, settings_wide], "shard2")
create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2")
create_tables_old_format('polymorphic_table', [node5, node6], "shard3")
create_tables('in_memory_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4")
create_tables('wal_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard4")
create_tables('restore_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard5")
create_tables('deduplication_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5")
create_tables('sync_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5")
create_tables('alters_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5")
yield cluster
@ -84,8 +103,8 @@ def start_cluster():
@pytest.mark.parametrize(
('first_node', 'second_node'),
[
(node1, node2),
(node5, node6)
(node1, node2), # compact parts
(node5, node6), # compact parts, old-format
]
)
def test_polymorphic_parts_basics(start_cluster, first_node, second_node):
@ -140,6 +159,31 @@ def test_polymorphic_parts_basics(start_cluster, first_node, second_node):
second_node.query("SELECT count(ss) FROM polymorphic_table") == "2000\n"
second_node.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n"
# Checks mostly that merge from compact part to compact part works.
def test_compact_parts_only(start_cluster):
for i in range(20):
insert_random_data('compact_parts_only', node1, 100)
insert_random_data('compact_parts_only', node2, 100)
node1.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20)
node2.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20)
assert node1.query("SELECT count() FROM compact_parts_only") == "4000\n"
assert node2.query("SELECT count() FROM compact_parts_only") == "4000\n"
assert node1.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'compact_parts_only' AND active") == "Compact\n"
assert node2.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'compact_parts_only' AND active") == "Compact\n"
node1.query("OPTIMIZE TABLE compact_parts_only FINAL")
node2.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20)
assert node2.query("SELECT count() FROM compact_parts_only") == "4000\n"
expected = "Compact\t1\n"
assert TSV(node1.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'compact_parts_only' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
assert TSV(node2.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'compact_parts_only' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
# Check that follower replicas create parts of the same type, which leader has chosen at merge.
@pytest.mark.parametrize(
@ -173,8 +217,8 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type):
node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True)
node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True)
settings7 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760}
settings8 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0}
settings7 = {'index_granularity_bytes' : 10485760}
settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
@pytest.fixture(scope="module")
def start_cluster_diff_versions():
@ -187,7 +231,7 @@ def start_cluster_diff_versions():
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes}
SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}
'''.format(name=name, **settings7)
)
@ -197,7 +241,7 @@ def start_cluster_diff_versions():
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes},
SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes},
min_rows_for_wide_part = {min_rows_for_wide_part}, min_bytes_for_wide_part = {min_bytes_for_wide_part}
'''.format(name=name, **settings8)
)
@ -263,6 +307,161 @@ def test_polymorphic_parts_non_adaptive(start_cluster):
assert node1.contains_in_log("<Warning> default.non_adaptive_table: Table can't create parts with adaptive granularity")
def test_in_memory(start_cluster):
node9.query("SYSTEM STOP MERGES")
node10.query("SYSTEM STOP MERGES")
for size in [200, 200, 300, 600]:
insert_random_data('in_memory_table', node9, size)
node10.query("SYSTEM SYNC REPLICA in_memory_table", timeout=20)
assert node9.query("SELECT count() FROM in_memory_table") == "1300\n"
assert node10.query("SELECT count() FROM in_memory_table") == "1300\n"
expected = "Compact\t1\nInMemory\t2\nWide\t1\n"
assert TSV(node9.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
node9.query("SYSTEM START MERGES")
node10.query("SYSTEM START MERGES")
assert_eq_with_retry(node9, "OPTIMIZE TABLE in_memory_table FINAL SETTINGS optimize_throw_if_noop = 1", "")
node10.query("SYSTEM SYNC REPLICA in_memory_table", timeout=20)
assert node9.query("SELECT count() FROM in_memory_table") == "1300\n"
assert node10.query("SELECT count() FROM in_memory_table") == "1300\n"
assert TSV(node9.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV("Wide\t1\n")
assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV("Wide\t1\n")
def test_in_memory_wal(start_cluster):
# Merges are disabled in config
for i in range(5):
insert_random_data('wal_table', node11, 50)
node12.query("SYSTEM SYNC REPLICA wal_table", timeout=20)
def check(node, rows, parts):
node.query("SELECT count() FROM wal_table") == "{}\n".format(rows)
node.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == "{}\n".format(parts)
check(node11, 250, 5)
check(node12, 250, 5)
# WAL works at inserts
node11.restart_clickhouse(kill=True)
check(node11, 250, 5)
# WAL works at fetches
node12.restart_clickhouse(kill=True)
check(node12, 250, 5)
insert_random_data('wal_table', node11, 50)
node12.query("SYSTEM SYNC REPLICA wal_table", timeout=20)
# Disable replication
with PartitionManager() as pm:
pm.partition_instances(node11, node12)
check(node11, 300, 6)
wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal.bin")
# Corrupt wal file
open(wal_file, 'rw+').truncate(os.path.getsize(wal_file) - 10)
node11.restart_clickhouse(kill=True)
# Broken part is lost, but other restored successfully
check(node11, 250, 5)
# WAL with blocks from 0 to 4
broken_wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal_0_4.bin")
assert os.path.exists(broken_wal_file)
# Fetch lost part from replica
node11.query("SYSTEM SYNC REPLICA wal_table", timeout=20)
check(node11, 300, 6)
#Check that new data is written to new wal, but old is still exists for restoring
assert os.path.getsize(wal_file) > 0
assert os.path.exists(broken_wal_file)
# Data is lost without WAL
node11.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0")
with PartitionManager() as pm:
pm.partition_instances(node11, node12)
insert_random_data('wal_table', node11, 50)
check(node11, 350, 7)
node11.restart_clickhouse(kill=True)
check(node11, 300, 6)
def test_in_memory_wal_rotate(start_cluster):
# Write every part to single wal
node11.query("ALTER TABLE restore_table MODIFY SETTING write_ahead_log_max_bytes = 10")
for i in range(5):
insert_random_data('restore_table', node11, 50)
for i in range(5):
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i))
assert os.path.exists(wal_file)
for node in [node11, node12]:
node.query("ALTER TABLE restore_table MODIFY SETTING number_of_free_entries_in_pool_to_lower_max_size_of_merge = 0")
node.query("ALTER TABLE restore_table MODIFY SETTING max_bytes_to_merge_at_max_space_in_pool = 10000000")
assert_eq_with_retry(node11, "OPTIMIZE TABLE restore_table FINAL SETTINGS optimize_throw_if_noop = 1", "")
# Restart to be sure, that clearing stale logs task was ran
node11.restart_clickhouse(kill=True)
for i in range(5):
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i))
assert not os.path.exists(wal_file)
# New wal file was created and ready to write part to it
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal.bin")
assert os.path.exists(wal_file)
assert os.path.getsize(wal_file) == 0
def test_in_memory_deduplication(start_cluster):
for i in range(3):
node9.query("INSERT INTO deduplication_table (date, id, s) VALUES (toDate('2020-03-03'), 1, 'foo')")
node10.query("INSERT INTO deduplication_table (date, id, s) VALUES (toDate('2020-03-03'), 1, 'foo')")
node9.query("SYSTEM SYNC REPLICA deduplication_table", timeout=20)
node10.query("SYSTEM SYNC REPLICA deduplication_table", timeout=20)
assert node9.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n"
assert node10.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n"
# Checks that restoring from WAL works after table schema changed
def test_in_memory_alters(start_cluster):
def check_parts_type(parts_num):
assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'alters_table' \
AND active GROUP BY part_type") == "InMemory\t{}\n".format(parts_num)
node9.query("INSERT INTO alters_table (date, id, s) VALUES (toDate('2020-10-10'), 1, 'ab'), (toDate('2020-10-10'), 2, 'cd')")
node9.query("ALTER TABLE alters_table ADD COLUMN col1 UInt32")
node9.restart_clickhouse(kill=True)
expected = "1\tab\t0\n2\tcd\t0\n"
assert node9.query("SELECT id, s, col1 FROM alters_table") == expected
check_parts_type(1)
node9.query("INSERT INTO alters_table (date, id, col1) VALUES (toDate('2020-10-10'), 3, 100)")
node9.query("ALTER TABLE alters_table MODIFY COLUMN col1 String")
node9.query("ALTER TABLE alters_table DROP COLUMN s")
node9.restart_clickhouse(kill=True)
check_parts_type(2)
with pytest.raises(Exception):
node9.query("SELECT id, s, col1 FROM alters_table")
expected = expected = "1\t0_foo\n2\t0_foo\n3\t100_foo\n"
assert node9.query("SELECT id, col1 || '_foo' FROM alters_table")
def test_polymorphic_parts_index(start_cluster):
node1.query('''

View File

@ -13,6 +13,13 @@
SAMPLE BY intHash32(UserID)
SETTINGS min_bytes_for_wide_part = '10M'
</create_query>
<create_query>
CREATE TABLE hits_memory AS hits_10m_single ENGINE = MergeTree()
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS min_bytes_for_compact_part = '1M', min_bytes_for_wide_part = '10M', in_memory_parts_enable_wal = 1
</create_query>
<create_query>
CREATE TABLE hits_buffer AS hits_10m_single
ENGINE = Buffer(default, hits_wide, 1, 0, 0, 10000, 10000, 0, 0)
@ -31,5 +38,6 @@
<drop_query>DROP TABLE IF EXISTS hits_wide</drop_query>
<drop_query>DROP TABLE IF EXISTS hits_compact</drop_query>
<drop_query>DROP TABLE IF EXISTS hits_memory</drop_query>
<drop_query>DROP TABLE IF EXISTS hits_buffer</drop_query>
</test>

View File

@ -0,0 +1,39 @@
system.parts
InMemory 2
1
1
Simple selects
0 0
1 1
2 2
3 0
4 1
50 2
51 0
52 1
53 2
54 0
34
0
20
10
Mutations and Alters
66
1 1
2 2
4 1
5 2
7 1
[1,1]
[]
[4,16]
[]
[7,49]
1 1
2 1
1 [1,1]
2 []
4 [4,16]
5 []
7 [7,49]
0

View File

@ -0,0 +1,46 @@
DROP TABLE IF EXISTS in_memory;
CREATE TABLE in_memory (a UInt32, b UInt32)
ENGINE = MergeTree ORDER BY a
SETTINGS min_rows_for_compact_part = 1000, min_rows_for_compact_part = 1000;
INSERT INTO in_memory SELECT number, number % 3 FROM numbers(100);
SELECT 'system.parts';
SELECT DISTINCT part_type, marks FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active;
SELECT DISTINCT data_uncompressed_bytes > 0 FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active;
SELECT DISTINCT column_data_uncompressed_bytes > 0 FROM system.parts_columns WHERE database = currentDatabase() AND table = 'in_memory' AND active;
SELECT 'Simple selects';
SELECT * FROM in_memory ORDER BY a LIMIT 5;
SELECT * FROM in_memory ORDER BY a LIMIT 5 OFFSET 50;
SELECT count() FROM in_memory WHERE b = 0 SETTINGS max_block_size = 10;
-- Check index
SELECT count() FROM in_memory WHERE a > 100 SETTINGS max_rows_to_read = 0, force_primary_key = 1;
SELECT count() FROM in_memory WHERE a >= 10 AND a < 30 SETTINGS force_primary_key = 1;
SELECT DISTINCT blockSize() FROM in_memory SETTINGS max_block_size = 10;
SELECT 'Mutations and Alters';
SET mutations_sync = 1;
ALTER TABLE in_memory DELETE WHERE b = 0;
SELECT count() FROM in_memory;
SELECT * FROM in_memory ORDER BY a LIMIT 5;
ALTER TABLE in_memory ADD COLUMN arr Array(UInt64);
ALTER TABLE in_memory UPDATE arr = [a, a * a] WHERE b = 1;
SELECT arr FROM in_memory ORDER BY a LIMIT 5;
ALTER TABLE in_memory MODIFY COLUMN b String;
ALTER TABLE in_memory RENAME COLUMN b to str;
SELECT DISTINCT str, length(str) FROM in_memory ORDER BY str;
ALTER TABLE in_memory DROP COLUMN str;
SELECT * FROM in_memory ORDER BY a LIMIT 5;
-- in-memory parts works if they're empty.
ALTER TABLE in_memory DELETE WHERE 1;
SELECT count() FROM in_memory;
DROP TABLE in_memory;

View File

@ -0,0 +1 @@
201901_1_1_0 1

View File

@ -0,0 +1,10 @@
-- Part of 00961_check_table test, but with in-memory parts
SET check_query_single_value_result = 0;
DROP TABLE IF EXISTS mt_table;
CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key
SETTINGS min_rows_for_compact_part = 1000, min_rows_for_compact_part = 1000;
CHECK TABLE mt_table;
INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World');
CHECK TABLE mt_table;
DROP TABLE mt_table;

View File

@ -0,0 +1 @@
0 bbb_aaa

View File

@ -0,0 +1,20 @@
-- Test 01266_default_prewhere_reqq, but with in-memory parts
DROP TABLE IF EXISTS t1;
CREATE TABLE t1
(
date Date,
s1 String,
s2 String
) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(date) ORDER BY (date, s1)
SETTINGS index_granularity = 8192, min_rows_for_compact_part = 1000, min_rows_for_wide_part = 1000;
set max_threads=1;
insert into t1 (date, s1,s2) values(today()-1,'aaa','bbb');
alter table t1 add column s3 String DEFAULT concat(s2,'_',s1);
-- insert into t1 (date, s1,s2) values(today(),'aaa2','bbb2');
select ignore(date), s3 from t1 where s2='bbb';
DROP TABLE t1;

View File

@ -0,0 +1,20 @@
[0]
[0,0,0]
[0,0,0,0,0]
[0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0]
[0]
[0,0,0]
[0,0,0,0,0]
[0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0]
[0]
[0,2,4]
[0,2,4,6,8]
[0,2,4,6,8,10,12]
[0,2,4,6,8,10,12,14,16]
[0] [0]
[0,1,2] [0,2,4]
[0,1,2,3,4] [0,2,4,6,8]
[0,1,2,3,4,5,6] [0,2,4,6,8,10,12]
[0,1,2,3,4,5,6,7,8] [0,2,4,6,8,10,12,14,16]

View File

@ -0,0 +1,17 @@
-- Test 00576_nested_and_prewhere, but with in-memory parts.
DROP TABLE IF EXISTS nested;
CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeTree ORDER BY x
SETTINGS min_rows_for_compact_part = 200000, min_rows_for_wide_part = 300000;
INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000;
ALTER TABLE nested ADD COLUMN n.b Array(UInt64);
SELECT DISTINCT n.b FROM nested PREWHERE filter;
SELECT DISTINCT n.b FROM nested PREWHERE filter SETTINGS max_block_size = 10;
ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a);
SELECT DISTINCT n.c FROM nested PREWHERE filter;
SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter;
DROP TABLE nested;

View File

@ -0,0 +1,36 @@
1 2 foo
1 3 bar
2 4 aa
2 5 bb
3 6 qq
3 7 ww
==================
2 4 aa
2 5 bb
3 6 qq
3 7 ww
==================
3 6 qq
3 7 ww
==================
2 4 aa
2 5 bb
3 6 qq
3 7 ww
2_4_4_0 Compact
3_3_3_0 InMemory
==================
2 4 aa
2 5 bb
3 6 qq
3 7 ww
==================
2 4 aa
2 5 bb
3 6 cc
3 7 dd
t2 2_4_4_0 Compact
t2 3_6_6_0 Compact
t3 3_1_1_0 InMemory
==================
3_1_1_0 InMemory 1

View File

@ -0,0 +1,52 @@
DROP TABLE IF EXISTS t2;
CREATE TABLE t2(id UInt32, a UInt64, s String)
ENGINE = MergeTree ORDER BY a PARTITION BY id
SETTINGS min_rows_for_compact_part = 1000, min_rows_for_wide_part = 2000;
SYSTEM STOP MERGES t2;
INSERT INTO t2 VALUES (1, 2, 'foo'), (1, 3, 'bar');
INSERT INTO t2 VALUES (2, 4, 'aa'), (2, 5, 'bb');
INSERT INTO t2 VALUES (3, 6, 'qq'), (3, 7, 'ww');
SELECT * FROM t2 ORDER BY a;
SELECT '==================';
ALTER TABLE t2 DROP PARTITION 1;
SELECT * FROM t2 ORDER BY a;
SELECT '==================';
ALTER TABLE t2 DETACH PARTITION 2;
SELECT * FROM t2 ORDER BY a;
SELECT '==================';
ALTER TABLE t2 ATTACH PARTITION 2;
SELECT * FROM t2 ORDER BY a;
SELECT name, part_type FROM system.parts WHERE table = 't2' AND active AND database = currentDatabase() ORDER BY name;
SELECT '==================';
DETACH TABLE t2;
ATTACH TABLE t2;
SELECT * FROM t2 ORDER BY a;
SELECT '==================';
DROP TABLE IF EXISTS t3;
CREATE TABLE t3(id UInt32, a UInt64, s String)
ENGINE = MergeTree ORDER BY a PARTITION BY id
SETTINGS min_rows_for_compact_part = 1000, min_rows_for_wide_part = 2000;
INSERT INTO t3 VALUES (3, 6, 'cc'), (3, 7, 'dd');
ALTER TABLE t2 REPLACE PARTITION 3 FROM t3;
SELECT * FROM t2 ORDER BY a;
SELECT table, name, part_type FROM system.parts WHERE table = 't2' AND active AND database = currentDatabase() ORDER BY name;
SELECT table, name, part_type FROM system.parts WHERE table = 't3' AND active AND database = currentDatabase() ORDER BY name;
SELECT '==================';
ALTER TABLE t3 FREEZE PARTITION 3;
SELECT name, part_type, is_frozen FROM system.parts WHERE table = 't3' AND active AND database = currentDatabase() ORDER BY name;
DROP TABLE t2;
DROP TABLE t3;