未验证 提交 05b10048 编写于 作者: A alesapin 提交者: GitHub

Merge pull request #14116 from ClickHouse/parts_default_compression

Save parts default compression
......@@ -106,6 +106,15 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour
memcpy(dest, compressed_buf.data(), decompressed_size);
}
std::vector<uint8_t> CompressionCodecMultiple::getCodecsBytesFromData(const char * source)
{
std::vector<uint8_t> result;
uint8_t compression_methods_size = source[0];
for (size_t i = 0; i < compression_methods_size; ++i)
result.push_back(source[1 + i]);
return result;
}
bool CompressionCodecMultiple::isCompression() const
{
for (const auto & codec : codecs)
......
......@@ -17,6 +17,8 @@ public:
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
static std::vector<uint8_t> getCodecsBytesFromData(const char * source);
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
......
......@@ -5,6 +5,7 @@
#include <Parsers/ASTFunction.h>
#include <common/unaligned.h>
#include <Common/Exception.h>
#include <Parsers/queryToString.h>
namespace DB
......@@ -20,7 +21,17 @@ ASTPtr ICompressionCodec::getFullCodecDesc() const
{
std::shared_ptr<ASTFunction> result = std::make_shared<ASTFunction>();
result->name = "CODEC";
result->arguments = getCodecDesc();
ASTPtr codec_desc = getCodecDesc();
if (codec_desc->as<ASTExpressionList>())
{
result->arguments = codec_desc;
}
else
{
result->arguments = std::make_shared<ASTExpressionList>();
result->arguments->children.push_back(codec_desc);
}
result->children.push_back(result->arguments);
return result;
}
......
#include <Compression/getCompressionCodecForFile.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <IO/ReadBufferFromFileBase.h>
#include <Compression/CompressionCodecMultiple.h>
#include <Common/PODArray.h>
#include <common/logger_useful.h>
#include <Common/UInt128.h>
namespace DB
{
using Checksum = CityHash_v1_0_2::uint128;
CompressionCodecPtr getCompressionCodecForFile(const DiskPtr & disk, const String & relative_path)
{
auto read_buffer = disk->readFile(relative_path);
read_buffer->ignore(sizeof(Checksum));
UInt8 header_size = ICompressionCodec::getHeaderSize();
PODArray<char> compressed_buffer;
compressed_buffer.resize(header_size);
read_buffer->readStrict(compressed_buffer.data(), header_size);
uint8_t method = ICompressionCodec::readMethod(compressed_buffer.data());
if (method == static_cast<uint8_t>(CompressionMethodByte::Multiple))
{
compressed_buffer.resize(1);
read_buffer->readStrict(compressed_buffer.data(), 1);
compressed_buffer.resize(1 + compressed_buffer[0]);
read_buffer->readStrict(compressed_buffer.data() + 1, compressed_buffer[0]);
auto codecs_bytes = CompressionCodecMultiple::getCodecsBytesFromData(compressed_buffer.data());
Codecs codecs;
for (auto byte : codecs_bytes)
codecs.push_back(CompressionCodecFactory::instance().get(byte));
return std::make_shared<CompressionCodecMultiple>(codecs);
}
return CompressionCodecFactory::instance().get(method);
}
}
#pragma once
#include <Compression/ICompressionCodec.h>
#include <Disks/IDisk.h>
namespace DB
{
/// Return compression codec with default parameters for file compressed in
/// clickhouse fashion (with checksums, headers for each block, etc). This
/// method should be used as fallback when we cannot deduce compression codec
/// from metadata.
CompressionCodecPtr getCompressionCodecForFile(const DiskPtr & disk, const String & relative_path);
}
......@@ -27,6 +27,7 @@ SRCS(
CompressionCodecT64.cpp
CompressionCodecZSTD.cpp
CompressionFactory.cpp
getCompressionCodecForFile.cpp
ICompressionCodec.cpp
LZ4_decompress_faster.cpp
......
......@@ -404,6 +404,13 @@ std::optional<ColumnDefault> ColumnsDescription::getDefault(const String & colum
}
bool ColumnsDescription::hasCompressionCodec(const String & column_name) const
{
const auto it = columns.get<1>().find(column_name);
return it != columns.get<1>().end() && it->codec != nullptr;
}
CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const
{
const auto it = columns.get<1>().find(column_name);
......
......@@ -111,6 +111,8 @@ public:
bool hasDefault(const String & column_name) const;
std::optional<ColumnDefault> getDefault(const String & column_name) const;
/// Does column has non default specified compression codec
bool hasCompressionCodec(const String & column_name) const;
CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const;
CompressionCodecPtr getCodecOrDefault(const String & column_name) const;
......
......@@ -43,6 +43,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;
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION = 4;
std::string getEndpointId(const std::string & node_id)
......@@ -83,7 +84,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_TYPE))});
response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION))});
++total_sends;
SCOPE_EXIT({--total_sends;});
......@@ -115,7 +116,10 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
if (isInMemoryPart(part))
sendPartFromMemory(part, out);
else
sendPartFromDisk(part, out);
{
bool send_default_compression_file = client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION;
sendPartFromDisk(part, out, send_default_compression_file);
}
}
catch (const NetException &)
{
......@@ -147,14 +151,20 @@ void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteB
block_out.write(part_in_memory->block);
}
void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out)
void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, bool send_default_compression_file)
{
/// 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"] = {};
auto file_names_without_checksums = part->getFileNamesWithoutChecksums();
for (const auto & file_name : file_names_without_checksums)
{
if (!send_default_compression_file && file_name == IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME)
continue;
checksums.files[file_name] = {};
}
auto disk = part->volume->getDisk();
MergeTreeData::DataPart::Checksums data_checksums;
writeBinary(checksums.files.size(), out);
......@@ -162,7 +172,6 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf
{
String file_name = it.first;
auto disk = part->volume->getDisk();
String path = part->getFullRelativePath() + file_name;
UInt64 size = disk->getFileSize(path);
......@@ -182,8 +191,7 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf
writePODBinary(hashing_out.getHash(), out);
if (file_name != "checksums.txt" &&
file_name != "columns.txt")
if (!file_names_without_checksums.count(file_name))
data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash());
}
......@@ -230,7 +238,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
{
{"endpoint", getEndpointId(replica_path)},
{"part", part_name},
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE)},
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION)},
{"compress", "false"}
});
......@@ -308,7 +316,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
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);
MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
part_out.writePrefix();
part_out.write(block);
part_out.writeSuffixAndFinalizePart(new_data_part);
......@@ -381,7 +389,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
ErrorCodes::CHECKSUM_DOESNT_MATCH);
if (file_name != "checksums.txt" &&
file_name != "columns.txt")
file_name != "columns.txt" &&
file_name != IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME)
checksums.addFile(file_name, file_size, expected_hash);
}
......
......@@ -32,7 +32,7 @@ public:
private:
MergeTreeData::DataPartPtr findPart(const String & name);
void sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
void sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
void sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, bool send_default_compression_file);
private:
/// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish,
......
......@@ -13,6 +13,8 @@
#include <Common/escapeForFileName.h>
#include <common/JSON.h>
#include <common/logger_useful.h>
#include <Compression/getCompressionCodecForFile.h>
#include <Parsers/queryToString.h>
namespace DB
{
......@@ -31,10 +33,6 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
extern const char * DELETE_ON_DESTROY_MARKER_PATH;
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
{
return disk->readFile(path, std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), disk->getFileSize(path)));
......@@ -418,9 +416,10 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`.
loadPartitionAndMinMaxIndex();
loadTTLInfos();
if (check_consistency)
checkConsistency(require_columns_checksums);
loadDefaultCompressionCodec();
}
void IMergeTreeDataPart::loadIndexGranularity()
......@@ -474,6 +473,89 @@ void IMergeTreeDataPart::loadIndex()
}
}
NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const
{
if (!isStoredOnDisk())
return {};
NameSet result = {"checksums.txt", "columns.txt"};
String default_codec_path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME;
if (volume->getDisk()->exists(default_codec_path))
result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
return result;
}
void IMergeTreeDataPart::loadDefaultCompressionCodec()
{
/// In memory parts doesn't have any compression
if (!isStoredOnDisk())
{
default_codec = CompressionCodecFactory::instance().get("NONE", {});
return;
}
String path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME;
if (!volume->getDisk()->exists(path))
{
default_codec = detectDefaultCompressionCodec();
}
else
{
auto file_buf = openForReading(volume->getDisk(), path);
String codec_line;
readEscapedStringUntilEOL(codec_line, *file_buf);
ReadBufferFromString buf(codec_line);
if (!checkString("CODEC", buf))
{
LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced automatically, from data on disk", name, path, codec_line);
default_codec = detectDefaultCompressionCodec();
}
try
{
ParserCodec codec_parser;
auto codec_ast = parseQuery(codec_parser, codec_line.data() + buf.getPosition(), codec_line.data() + codec_line.length(), "codec parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
default_codec = CompressionCodecFactory::instance().get(codec_ast, {});
}
catch (const DB::Exception & ex)
{
LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}', error '{}'. Default compression codec will be deduced automatically, from data on disk.", name, path, codec_line, ex.what());
default_codec = detectDefaultCompressionCodec();
}
}
}
CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const
{
/// In memory parts doesn't have any compression
if (!isStoredOnDisk())
return CompressionCodecFactory::instance().get("NONE", {});
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
const auto & storage_columns = metadata_snapshot->getColumns();
CompressionCodecPtr result = nullptr;
for (const auto & part_column : columns)
{
/// It was compressed with default codec and it's not empty
auto column_size = getColumnSize(part_column.name, *part_column.type);
if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name))
{
result = getCompressionCodecForFile(volume->getDisk(), getFullRelativePath() + getFileNameForColumn(part_column) + ".bin");
break;
}
}
if (!result)
result = CompressionCodecFactory::instance().getDefaultCodec();
return result;
}
void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
{
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
......@@ -795,7 +877,9 @@ void IMergeTreeDataPart::remove() const
for (const auto & file : {"checksums.txt", "columns.txt"})
volume->getDisk()->remove(to + "/" + file);
volume->getDisk()->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_PATH);
volume->getDisk()->removeIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME);
volume->getDisk()->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);
volume->getDisk()->remove(to);
}
......@@ -850,7 +934,7 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const Storag
/// Backup is not recursive (max_level is 0), so do not copy inner directories
localBackup(volume->getDisk(), getFullRelativePath(), destination_path, 0);
volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_PATH);
volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);
}
void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservation) const
......@@ -867,7 +951,7 @@ void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservat
reserved_disk->createDirectory(path_to_clone);
volume->getDisk()->copy(getFullRelativePath(), reserved_disk, path_to_clone);
volume->getDisk()->removeIfExists(path_to_clone + "/" + DELETE_ON_DESTROY_MARKER_PATH);
volume->getDisk()->removeIfExists(path_to_clone + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);
}
void IMergeTreeDataPart::checkConsistencyBase() const
......
......@@ -288,6 +288,8 @@ public:
/// Columns with values, that all have been zeroed by expired ttl
NameSet expired_columns;
CompressionCodecPtr default_codec;
/// For data in RAM ('index')
UInt64 getIndexSizeInBytes() const;
UInt64 getIndexSizeInAllocatedBytes() const;
......@@ -330,7 +332,20 @@ public:
String getRelativePathForPrefix(const String & prefix) const;
/// Return set of metadat file names without checksums. For example,
/// columns.txt or checksums.txt itself.
NameSet getFileNamesWithoutChecksums() const;
/// File with compression codec name which was used to compress part columns
/// by default. Some columns may have their own compression codecs, but
/// default will be stored in this file.
static inline constexpr auto DEFAULT_COMPRESSION_CODEC_FILE_NAME = "default_compression_codec.txt";
static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME = "delete-on-destroy.txt";
protected:
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
ColumnSize total_columns_size;
......@@ -380,6 +395,15 @@ private:
void loadTTLInfos();
void loadPartitionAndMinMaxIndex();
/// Load default compression codec from file default_compression_codec.txt
/// if it not exists tries to deduce codec from compressed column without
/// any specifial compression.
void loadDefaultCompressionCodec();
/// Found column without specific compression and return codec
/// for this column with default parameters.
CompressionCodecPtr detectDefaultCompressionCodec() const;
};
using MergeTreeDataPartState = IMergeTreeDataPart::State;
......
......@@ -113,9 +113,6 @@ namespace ErrorCodes
}
const char * DELETE_ON_DESTROY_MARKER_PATH = "delete-on-destroy.txt";
static void checkSampleExpression(const StorageInMemoryMetadata & metadata, bool allow_sampling_expression_not_in_primary_key)
{
const auto & pk_sample_block = metadata.getPrimaryKey().sample_block;
......@@ -781,7 +778,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
bool broken = false;
String part_path = relative_data_path + "/" + part_name;
String marker_path = part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH;
String marker_path = part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME;
if (part_disk_ptr->exists(marker_path))
{
LOG_WARNING(log, "Detaching stale part {}{}, which should have been deleted after a move. That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", getFullPathOnDisk(part_disk_ptr), part_name);
......@@ -908,6 +905,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
for (auto & part : broken_parts_to_detach)
part->renameToDetached("");
/// Delete from the set of current parts those parts that are covered by another part (those parts that
/// were merged), but that for some reason are still not deleted from the filesystem.
/// Deletion of files will be performed later in the clearOldParts() method.
......@@ -958,6 +956,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
calculateColumnSizesImpl();
LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size());
}
......@@ -2370,7 +2369,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy)
modifyPartState(part_it, DataPartState::Committed);
auto disk = original_active_part->volume->getDisk();
String marker_path = original_active_part->getFullRelativePath() + DELETE_ON_DESTROY_MARKER_PATH;
String marker_path = original_active_part->getFullRelativePath() + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME;
try
{
disk->createFile(marker_path);
......@@ -2441,14 +2440,6 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
part->modification_time = disk->getLastModified(full_part_path).epochTime();
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const
{
MutableDataPartPtr part = createPart(Poco::Path(relative_path).getFileName(), volume, relative_path);
loadPartAndFixMetadataImpl(part);
return part;
}
void MergeTreeData::calculateColumnSizesImpl()
{
column_sizes.clear();
......@@ -2919,6 +2910,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
LOG_DEBUG(log, "Checking parts");
MutableDataPartsVector loaded_parts;
loaded_parts.reserve(renamed_parts.old_and_new_names.size());
for (const auto & part_names : renamed_parts.old_and_new_names)
{
LOG_DEBUG(log, "Checking part {}", part_names.second);
......@@ -3286,7 +3278,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
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);
disk->removeIfExists(dst_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk);
auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name);
......@@ -3377,7 +3369,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(MatcherFn m
else
localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path);
part->volume->getDisk()->removeIfExists(backup_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH);
part->volume->getDisk()->removeIfExists(backup_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
part->is_frozen.store(true, std::memory_order_relaxed);
result.push_back(PartitionCommandResultInfo{
......
......@@ -534,9 +534,6 @@ public:
return DB::extractKeyExpressionList(node);
}
/// Check that the part is not broken and calculate the checksums for it if they are not present.
MutableDataPartPtr loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const;
/** Create local backup (snapshot) for parts with specified prefix.
* 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.
......
......@@ -29,6 +29,7 @@
#include <Common/interpolate.h>
#include <Common/typeid_cast.h>
#include <Common/escapeForFileName.h>
#include <Parsers/queryToString.h>
#include <cmath>
#include <ctime>
......@@ -1122,7 +1123,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
/// We will modify only some of the columns. Other columns and key values can be copied as-is.
auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context);
NameSet files_to_skip = collectFilesToSkip(updated_header, indices_to_recalc, mrk_extension);
NameSet files_to_skip = collectFilesToSkip(source_part, updated_header, indices_to_recalc, mrk_extension);
NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension);
if (need_remove_expired_values)
......@@ -1184,7 +1185,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
}
}
finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values);
finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values, compression_codec);
}
return new_data_part;
......@@ -1448,9 +1449,12 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames(
}
NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension)
const MergeTreeDataPartPtr & source_part,
const Block & updated_header,
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
const String & mrk_extension)
{
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
NameSet files_to_skip = source_part->getFileNamesWithoutChecksums();
/// Skip updated files
for (const auto & entry : updated_header)
......@@ -1737,7 +1741,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns(
void MergeTreeDataMergerMutator::finalizeMutatedPart(
const MergeTreeDataPartPtr & source_part,
MergeTreeData::MutableDataPartPtr new_data_part,
bool need_remove_expired_values)
bool need_remove_expired_values,
const CompressionCodecPtr & codec)
{
auto disk = new_data_part->volume->getDisk();
if (need_remove_expired_values)
......@@ -1756,6 +1761,10 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart(
new_data_part->checksums.write(*out_checksums);
} /// close fd
{
auto out = disk->writeFile(new_data_part->getFullRelativePath() + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096);
DB::writeText(queryToString(codec->getFullCodecDesc()), *out);
}
{
/// Write a file with a description of columns.
......@@ -1770,6 +1779,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart(
new_data_part->modification_time = time(nullptr);
new_data_part->setBytesOnDisk(
MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath()));
new_data_part->default_codec = codec;
new_data_part->calculateColumnsSizesOnDisk();
}
......
......@@ -156,7 +156,11 @@ private:
/// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt.
/// Because we will generate new versions of them after we perform mutation.
static NameSet collectFilesToSkip(const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension);
static NameSet collectFilesToSkip(
const MergeTreeDataPartPtr & source_part,
const Block & updated_header,
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
const String & mrk_extension);
/// Get the columns list of the resulting part in the same order as storage_columns.
static NamesAndTypesList getColumnsForNewDataPart(
......@@ -209,7 +213,8 @@ private:
static void finalizeMutatedPart(
const MergeTreeDataPartPtr & source_part,
MergeTreeData::MutableDataPartPtr new_data_part,
bool need_remove_expired_values);
bool need_remove_expired_values,
const CompressionCodecPtr & codec);
public :
/** Is used to cancel all merges and mutations. On cancel() call all currently running actions will throw exception soon.
......
......@@ -56,7 +56,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
const NamesAndTypesList & columns_list,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const CompressionCodecPtr & default_codec,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) const
{
......@@ -71,7 +71,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
return std::make_unique<MergeTreeDataPartWriterCompact>(
shared_from_this(), ordered_columns_list, metadata_snapshot,
indices_to_recalc, index_granularity_info.marks_file_extension,
default_codec, writer_settings, computed_index_granularity);
default_codec_, writer_settings, computed_index_granularity);
}
......
......@@ -24,6 +24,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
const std::optional<String> & relative_path_)
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY)
{
default_codec = CompressionCodecFactory::instance().get("NONE", {});
}
MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
......@@ -34,6 +35,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
const std::optional<String> & relative_path_)
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY)
{
default_codec = CompressionCodecFactory::instance().get("NONE", {});
}
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader(
......@@ -129,5 +131,4 @@ DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part)
{
return std::dynamic_pointer_cast<const MergeTreeDataPartInMemory>(part);
}
}
......@@ -57,14 +57,14 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter(
const NamesAndTypesList & columns_list,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const CompressionCodecPtr & default_codec,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) const
{
return std::make_unique<MergeTreeDataPartWriterWide>(
shared_from_this(), columns_list, metadata_snapshot, indices_to_recalc,
index_granularity_info.marks_file_extension,
default_codec, writer_settings, computed_index_granularity);
default_codec_, writer_settings, computed_index_granularity);
}
......
......@@ -151,7 +151,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
if (action_type == ActionType::ADD_PART)
{
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, nullptr);
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
part->minmax_idx.update(block, storage.minmax_idx_columns);
part->partition.create(metadata_snapshot, block, 0);
......
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <Parsers/queryToString.h>
namespace DB
......@@ -18,14 +19,14 @@ MergedBlockOutputStream::MergedBlockOutputStream(
const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices,
CompressionCodecPtr default_codec,
CompressionCodecPtr default_codec_,
bool blocks_are_granules_size)
: MergedBlockOutputStream(
data_part,
metadata_snapshot_,
columns_list_,
skip_indices,
default_codec,
default_codec_,
{},
data_part->storage.global_context.getSettings().min_bytes_to_use_direct_io,
blocks_are_granules_size)
......@@ -37,12 +38,13 @@ MergedBlockOutputStream::MergedBlockOutputStream(
const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices,
CompressionCodecPtr default_codec,
CompressionCodecPtr default_codec_,
const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size,
size_t aio_threshold,
bool blocks_are_granules_size)
: IMergedBlockOutputStream(data_part, metadata_snapshot_)
, columns_list(columns_list_)
, default_codec(default_codec_)
{
MergeTreeWriterSettings writer_settings(
storage.global_context.getSettings(),
......@@ -120,6 +122,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->index_granularity = writer->getIndexGranularity();
new_part->calculateColumnsSizesOnDisk();
if (default_codec != nullptr)
new_part->default_codec = default_codec;
}
void MergedBlockOutputStream::finalizePartOnDisk(
......@@ -162,6 +166,17 @@ void MergedBlockOutputStream::finalizePartOnDisk(
part_columns.writeText(*out);
}
if (default_codec != nullptr)
{
auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096);
DB::writeText(queryToString(default_codec->getFullCodecDesc()), *out);
}
else
{
throw Exception("Compression codec have to be specified for part on disk, empty for" + new_part->name
+ ". It is a bug.", ErrorCodes::LOGICAL_ERROR);
}
{
/// Write file with checksums.
auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096);
......
......@@ -18,7 +18,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices,
CompressionCodecPtr default_codec,
CompressionCodecPtr default_codec_,
bool blocks_are_granules_size = false);
MergedBlockOutputStream(
......@@ -26,7 +26,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices,
CompressionCodecPtr default_codec,
CompressionCodecPtr default_codec_,
const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size,
size_t aio_threshold,
bool blocks_are_granules_size = false);
......@@ -64,6 +64,7 @@ private:
NamesAndTypesList columns_list;
IMergeTreeDataPart::MinMaxIndex minmax_idx;
size_t rows_count = 0;
CompressionCodecPtr default_codec;
};
}
......@@ -49,6 +49,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
const String & full_relative_path,
const NamesAndTypesList & columns_list,
const MergeTreeDataPartType & part_type,
const NameSet & files_without_checksums,
bool require_checksums,
std::function<bool()> is_cancelled)
{
......@@ -135,7 +136,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
auto checksum_it = checksums_data.files.find(file_name);
/// Skip files that we already calculated. Also skip metadata files that are not checksummed.
if (checksum_it == checksums_data.files.end() && file_name != "checksums.txt" && file_name != "columns.txt")
if (checksum_it == checksums_data.files.end() && !files_without_checksums.count(file_name))
{
auto txt_checksum_it = checksum_files_txt.find(file_name);
if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0)
......@@ -183,6 +184,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
data_part->getFullRelativePath(),
data_part->getColumns(),
data_part->getType(),
data_part->getFileNamesWithoutChecksums(),
require_checksums,
is_cancelled);
}
......
......@@ -17,6 +17,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
const String & full_relative_path,
const NamesAndTypesList & columns_list,
const MergeTreeDataPartType & part_type,
const NameSet & files_without_checksums,
bool require_checksums,
std::function<bool()> is_cancelled = []{ return false; });
......
......@@ -10,6 +10,7 @@
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/VirtualColumnUtils.h>
#include <Databases/IDatabase.h>
#include <Parsers/queryToString.h>
#include <Common/hex.h>
namespace DB
......@@ -60,6 +61,8 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_)
{"move_ttl_info.expression", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"move_ttl_info.min", std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>())},
{"move_ttl_info.max", std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>())},
{"default_compression_codec", std::make_shared<DataTypeString>()},
}
)
{
......@@ -169,6 +172,8 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto
columns_[i++]->insert(min_array);
columns_[i++]->insert(max_array);
}
columns_[i++]->insert(queryToString(part->default_codec->getCodecDesc()));
}
}
......
......@@ -15,7 +15,7 @@
<min_part_size_ratio>0</min_part_size_ratio> <!-- The minimum size of the part relative to all the data in the table. -->
<!-- Which compression method to choose. -->
<method>lz4hc</method>
<level>10</level>
<level>5</level>
</case>
<case>
<!-- Conditions. All must be satisfied simultaneously. Some conditions may not be specified. -->
......
......@@ -8,7 +8,7 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/default_compression.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/default_compression.xml'], with_zookeeper=True)
node3 = cluster.add_instance('node3', main_configs=['configs/default_compression.xml'], image='yandex/clickhouse-server:20.3.16', stay_alive=True, with_installed_binary=True)
@pytest.fixture(scope="module")
def start_cluster():
......@@ -58,21 +58,59 @@ def test_default_codec_single(start_cluster):
node1.query("INSERT INTO compression_table VALUES (2, '{}')".format(get_random_string(2048)))
# ZSTD(10) and LZ4 after merge
node1.query("INSERT INTO compression_table VALUES (3, '{}')".format(get_random_string(12048)))
node1.query("INSERT INTO compression_table VALUES (3, '{}')".format(get_random_string(22048)))
node2.query("SYSTEM SYNC REPLICA compression_table", timeout=15)
# to reload parts
node1.query("DETACH TABLE compression_table")
node2.query("DETACH TABLE compression_table")
node1.query("ATTACH TABLE compression_table")
node2.query("ATTACH TABLE compression_table")
node1.query("SYSTEM FLUSH LOGS")
node2.query("SYSTEM FLUSH LOGS")
# Same codec for all
assert get_compression_codec_byte(node1, "compression_table", "1_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "2_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "3_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
# just to be sure that replication works
node1.query("OPTIMIZE TABLE compression_table FINAL")
node2.query("SYSTEM SYNC REPLICA compression_table", timeout=15)
node1.query("OPTIMIZE TABLE compression_table FINAL")
# to reload parts
node1.query("DETACH TABLE compression_table")
node2.query("DETACH TABLE compression_table")
node1.query("ATTACH TABLE compression_table")
node2.query("ATTACH TABLE compression_table")
node1.query("SYSTEM FLUSH LOGS")
node2.query("SYSTEM FLUSH LOGS")
assert get_compression_codec_byte(node1, "compression_table", "1_0_0_1") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert get_compression_codec_byte(node1, "compression_table", "3_0_0_1") == CODECS_MAPPING['LZ4']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query("SELECT COUNT() FROM compression_table") == "3\n"
assert node2.query("SELECT COUNT() FROM compression_table") == "3\n"
......@@ -94,15 +132,23 @@ def test_default_codec_multiple(start_cluster):
node1.query("INSERT INTO compression_table_multiple VALUES (2, '{}')".format(get_random_string(2048)))
# ZSTD(10) and LZ4 after merge
node1.query("INSERT INTO compression_table_multiple VALUES (3, '{}')".format(get_random_string(12048)))
node1.query("INSERT INTO compression_table_multiple VALUES (3, '{}')".format(get_random_string(22048)))
# Same codec for all
assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15)
......@@ -110,6 +156,46 @@ def test_default_codec_multiple(start_cluster):
assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['LZ4']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query("SELECT COUNT() FROM compression_table_multiple") == "3\n"
assert node2.query("SELECT COUNT() FROM compression_table_multiple") == "3\n"
def test_default_codec_version_update(start_cluster):
node3.query("""
CREATE TABLE compression_table (
key UInt64 CODEC(LZ4HC(7)),
data1 String
) ENGINE = MergeTree ORDER BY tuple() PARTITION BY key;
""")
node3.query("INSERT INTO compression_table VALUES (1, 'x')")
node3.query("INSERT INTO compression_table VALUES (2, '{}')".format(get_random_string(2048)))
node3.query("INSERT INTO compression_table VALUES (3, '{}')".format(get_random_string(22048)))
node3.restart_with_latest_version()
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_0'") == "ZSTD(1)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_0'") == "ZSTD(1)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_0'") == "ZSTD(1)\n"
node3.query("OPTIMIZE TABLE compression_table FINAL")
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_1'") == "ZSTD(10)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_1'") == "LZ4HC(5)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_1'") == "LZ4\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['Multiple']
......
......@@ -26,8 +26,8 @@ def cluster():
FILES_OVERHEAD = 1
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6
FILES_OVERHEAD_PER_PART_COMPACT = 10
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
def random_string(length):
......@@ -55,7 +55,7 @@ def create_table(cluster, table_name, additional_settings=None):
ORDER BY (dt, id)
SETTINGS
storage_policy='s3',
old_parts_lifetime=0,
old_parts_lifetime=0,
index_granularity=512
""".format(table_name)
......
......@@ -84,6 +84,8 @@ def partition_complex_assert_checksums():
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0/columns.txt\n" \
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0/p.bin\n" \
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0/primary.idx\n" \
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition/19700102_2_2_0/default_compression_codec.txt\n" \
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition/19700201_1_1_0/default_compression_codec.txt\n" \
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0/count.txt\n" \
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0/count.txt\n" \
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0/p.bin\n" \
......
......@@ -30,7 +30,7 @@ def cluster():
FILES_OVERHEAD = 1
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
FILES_OVERHEAD_PER_PART = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6
FILES_OVERHEAD_PER_PART = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
def random_string(length):
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册