mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #29896 from kitaisreal/system-data-skipping-indices-added-size
System table data skipping indices added size
This commit is contained in:
commit
91a48e9651
@ -10,6 +10,9 @@ Columns:
|
|||||||
- `type` ([String](../../sql-reference/data-types/string.md)) — Index type.
|
- `type` ([String](../../sql-reference/data-types/string.md)) — Index type.
|
||||||
- `expr` ([String](../../sql-reference/data-types/string.md)) — Expression for the index calculation.
|
- `expr` ([String](../../sql-reference/data-types/string.md)) — Expression for the index calculation.
|
||||||
- `granularity` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of granules in the block.
|
- `granularity` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of granules in the block.
|
||||||
|
- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of compressed data, in bytes.
|
||||||
|
- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of decompressed data, in bytes.
|
||||||
|
- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of marks, in bytes.
|
||||||
|
|
||||||
**Example**
|
**Example**
|
||||||
|
|
||||||
@ -26,6 +29,9 @@ name: clicks_idx
|
|||||||
type: minmax
|
type: minmax
|
||||||
expr: clicks
|
expr: clicks
|
||||||
granularity: 1
|
granularity: 1
|
||||||
|
data_compressed_bytes: 58
|
||||||
|
data_uncompressed_bytes: 6
|
||||||
|
marks: 48
|
||||||
|
|
||||||
Row 2:
|
Row 2:
|
||||||
──────
|
──────
|
||||||
@ -35,4 +41,7 @@ name: contacts_null_idx
|
|||||||
type: minmax
|
type: minmax
|
||||||
expr: assumeNotNull(contacts_null)
|
expr: assumeNotNull(contacts_null)
|
||||||
granularity: 1
|
granularity: 1
|
||||||
|
data_compressed_bytes: 58
|
||||||
|
data_uncompressed_bytes: 6
|
||||||
|
marks: 48
|
||||||
```
|
```
|
||||||
|
@ -38,6 +38,12 @@ Columns:
|
|||||||
|
|
||||||
- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks.
|
- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks.
|
||||||
|
|
||||||
|
- `secondary_indices_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of compressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included.
|
||||||
|
|
||||||
|
- `secondary_indices_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of uncompressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included.
|
||||||
|
|
||||||
|
- `secondary_indices_marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks for secondary indices.
|
||||||
|
|
||||||
- `modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.
|
- `modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.
|
||||||
|
|
||||||
- `remove_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time when the data part became inactive.
|
- `remove_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time when the data part became inactive.
|
||||||
@ -119,6 +125,9 @@ rows: 6
|
|||||||
bytes_on_disk: 310
|
bytes_on_disk: 310
|
||||||
data_compressed_bytes: 157
|
data_compressed_bytes: 157
|
||||||
data_uncompressed_bytes: 91
|
data_uncompressed_bytes: 91
|
||||||
|
secondary_indices_compressed_bytes: 58
|
||||||
|
secondary_indices_uncompressed_bytes: 6
|
||||||
|
secondary_indices_marks_bytes: 48
|
||||||
marks_bytes: 144
|
marks_bytes: 144
|
||||||
modification_time: 2020-06-18 13:01:49
|
modification_time: 2020-06-18 13:01:49
|
||||||
remove_time: 1970-01-01 00:00:00
|
remove_time: 1970-01-01 00:00:00
|
||||||
|
@ -87,6 +87,8 @@ struct ColumnSize
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
using IndexSize = ColumnSize;
|
||||||
|
|
||||||
/** Storage. Describes the table. Responsible for
|
/** Storage. Describes the table. Responsible for
|
||||||
* - storage of the table data;
|
* - storage of the table data;
|
||||||
* - the definition in which files (or not in files) the data is stored;
|
* - the definition in which files (or not in files) the data is stored;
|
||||||
@ -163,6 +165,11 @@ public:
|
|||||||
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||||
virtual ColumnSizeByName getColumnSizes() const { return {}; }
|
virtual ColumnSizeByName getColumnSizes() const { return {}; }
|
||||||
|
|
||||||
|
/// Optional size information of each secondary index.
|
||||||
|
/// Valid only for MergeTree family.
|
||||||
|
using IndexSizeByName = std::unordered_map<std::string, IndexSize>;
|
||||||
|
virtual IndexSizeByName getSecondaryIndexSizes() const { return {}; }
|
||||||
|
|
||||||
/// Get mutable version (snapshot) of storage metadata. Metadata object is
|
/// Get mutable version (snapshot) of storage metadata. Metadata object is
|
||||||
/// multiversion, so it can be concurrently changed, but returned copy can be
|
/// multiversion, so it can be concurrently changed, but returned copy can be
|
||||||
/// used without any locks.
|
/// used without any locks.
|
||||||
|
@ -584,7 +584,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
|
|||||||
loadColumns(require_columns_checksums);
|
loadColumns(require_columns_checksums);
|
||||||
loadChecksums(require_columns_checksums);
|
loadChecksums(require_columns_checksums);
|
||||||
loadIndexGranularity();
|
loadIndexGranularity();
|
||||||
calculateColumnsSizesOnDisk();
|
calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||||
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
|
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
|
||||||
loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`.
|
loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`.
|
||||||
loadPartitionAndMinMaxIndex();
|
loadPartitionAndMinMaxIndex();
|
||||||
@ -1420,6 +1420,11 @@ void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) cons
|
|||||||
throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk()
|
||||||
|
{
|
||||||
|
calculateColumnsSizesOnDisk();
|
||||||
|
calculateSecondaryIndicesSizesOnDisk();
|
||||||
|
}
|
||||||
|
|
||||||
void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
|
void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
|
||||||
{
|
{
|
||||||
@ -1429,6 +1434,41 @@ void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
|
|||||||
calculateEachColumnSizes(columns_sizes, total_columns_size);
|
calculateEachColumnSizes(columns_sizes, total_columns_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void IMergeTreeDataPart::calculateSecondaryIndicesSizesOnDisk()
|
||||||
|
{
|
||||||
|
if (checksums.empty())
|
||||||
|
throw Exception("Cannot calculate secondary indexes sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto secondary_indices_descriptions = storage.getInMemoryMetadataPtr()->secondary_indices;
|
||||||
|
|
||||||
|
for (auto & index_description : secondary_indices_descriptions)
|
||||||
|
{
|
||||||
|
ColumnSize index_size;
|
||||||
|
|
||||||
|
auto index_ptr = MergeTreeIndexFactory::instance().get(index_description);
|
||||||
|
auto index_name = index_ptr->getFileName();
|
||||||
|
auto index_name_escaped = escapeForFileName(index_name);
|
||||||
|
|
||||||
|
auto index_file_name = index_name_escaped + index_ptr->getSerializedFileExtension();
|
||||||
|
auto index_marks_file_name = index_name_escaped + index_granularity_info.marks_file_extension;
|
||||||
|
|
||||||
|
/// If part does not contain index
|
||||||
|
auto bin_checksum = checksums.files.find(index_file_name);
|
||||||
|
if (bin_checksum != checksums.files.end())
|
||||||
|
{
|
||||||
|
index_size.data_compressed = bin_checksum->second.file_size;
|
||||||
|
index_size.data_uncompressed = bin_checksum->second.uncompressed_size;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto mrk_checksum = checksums.files.find(index_marks_file_name);
|
||||||
|
if (mrk_checksum != checksums.files.end())
|
||||||
|
index_size.marks = mrk_checksum->second.file_size;
|
||||||
|
|
||||||
|
total_secondary_indices_size.add(index_size);
|
||||||
|
secondary_index_sizes[index_description.name] = index_size;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const
|
ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const
|
||||||
{
|
{
|
||||||
/// For some types of parts columns_size maybe not calculated
|
/// For some types of parts columns_size maybe not calculated
|
||||||
@ -1439,6 +1479,15 @@ ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const I
|
|||||||
return ColumnSize{};
|
return ColumnSize{};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_index_name) const
|
||||||
|
{
|
||||||
|
auto it = secondary_index_sizes.find(secondary_index_name);
|
||||||
|
if (it != secondary_index_sizes.end())
|
||||||
|
return it->second;
|
||||||
|
|
||||||
|
return ColumnSize{};
|
||||||
|
}
|
||||||
|
|
||||||
void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const
|
void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const
|
||||||
{
|
{
|
||||||
for (const auto & [column_name, size] : columns_sizes)
|
for (const auto & [column_name, size] : columns_sizes)
|
||||||
|
@ -55,6 +55,8 @@ public:
|
|||||||
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||||
using NameToNumber = std::unordered_map<std::string, size_t>;
|
using NameToNumber = std::unordered_map<std::string, size_t>;
|
||||||
|
|
||||||
|
using IndexSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||||
|
|
||||||
using Type = MergeTreeDataPartType;
|
using Type = MergeTreeDataPartType;
|
||||||
|
|
||||||
|
|
||||||
@ -101,9 +103,16 @@ public:
|
|||||||
/// Otherwise return information about column size on disk.
|
/// Otherwise return information about column size on disk.
|
||||||
ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const;
|
ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const;
|
||||||
|
|
||||||
|
/// NOTE: Returns zeros if secondary indexes are not found in checksums.
|
||||||
|
/// Otherwise return information about secondary index size on disk.
|
||||||
|
IndexSize getSecondaryIndexSize(const String & secondary_index_name) const;
|
||||||
|
|
||||||
/// Return information about column size on disk for all columns in part
|
/// Return information about column size on disk for all columns in part
|
||||||
ColumnSize getTotalColumnsSize() const { return total_columns_size; }
|
ColumnSize getTotalColumnsSize() const { return total_columns_size; }
|
||||||
|
|
||||||
|
/// Return information about secondary indexes size on disk for all indexes in part
|
||||||
|
IndexSize getTotalSeconaryIndicesSize() const { return total_secondary_indices_size; }
|
||||||
|
|
||||||
virtual String getFileNameForColumn(const NameAndTypePair & column) const = 0;
|
virtual String getFileNameForColumn(const NameAndTypePair & column) const = 0;
|
||||||
|
|
||||||
virtual ~IMergeTreeDataPart();
|
virtual ~IMergeTreeDataPart();
|
||||||
@ -341,7 +350,9 @@ public:
|
|||||||
|
|
||||||
/// Calculate the total size of the entire directory with all the files
|
/// Calculate the total size of the entire directory with all the files
|
||||||
static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from);
|
static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from);
|
||||||
void calculateColumnsSizesOnDisk();
|
|
||||||
|
/// Calculate column and secondary indices sizes on disk.
|
||||||
|
void calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||||
|
|
||||||
String getRelativePathForPrefix(const String & prefix) const;
|
String getRelativePathForPrefix(const String & prefix) const;
|
||||||
|
|
||||||
@ -396,6 +407,10 @@ protected:
|
|||||||
/// Size for each column, calculated once in calcuateColumnSizesOnDisk
|
/// Size for each column, calculated once in calcuateColumnSizesOnDisk
|
||||||
ColumnSizeByName columns_sizes;
|
ColumnSizeByName columns_sizes;
|
||||||
|
|
||||||
|
ColumnSize total_secondary_indices_size;
|
||||||
|
|
||||||
|
IndexSizeByName secondary_index_sizes;
|
||||||
|
|
||||||
/// Total size on disk, not only columns. May not contain size of
|
/// Total size on disk, not only columns. May not contain size of
|
||||||
/// checksums.txt and columns.txt. 0 - if not counted;
|
/// checksums.txt and columns.txt. 0 - if not counted;
|
||||||
UInt64 bytes_on_disk{0};
|
UInt64 bytes_on_disk{0};
|
||||||
@ -450,6 +465,10 @@ private:
|
|||||||
|
|
||||||
void loadPartitionAndMinMaxIndex();
|
void loadPartitionAndMinMaxIndex();
|
||||||
|
|
||||||
|
void calculateColumnsSizesOnDisk();
|
||||||
|
|
||||||
|
void calculateSecondaryIndicesSizesOnDisk();
|
||||||
|
|
||||||
/// Load default compression codec from file default_compression_codec.txt
|
/// Load default compression codec from file default_compression_codec.txt
|
||||||
/// if it not exists tries to deduce codec from compressed column without
|
/// if it not exists tries to deduce codec from compressed column without
|
||||||
/// any specifial compression.
|
/// any specifial compression.
|
||||||
|
@ -1167,7 +1167,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
calculateColumnSizesImpl();
|
calculateColumnAndSecondaryIndexSizesImpl();
|
||||||
|
|
||||||
|
|
||||||
LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size());
|
LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size());
|
||||||
@ -2352,7 +2352,7 @@ bool MergeTreeData::renameTempPartAndReplace(
|
|||||||
{
|
{
|
||||||
covered_part->remove_time.store(current_time, std::memory_order_relaxed);
|
covered_part->remove_time.store(current_time, std::memory_order_relaxed);
|
||||||
modifyPartState(covered_part, DataPartState::Outdated);
|
modifyPartState(covered_part, DataPartState::Outdated);
|
||||||
removePartContributionToColumnSizes(covered_part);
|
removePartContributionToColumnAndSecondaryIndexSizes(covered_part);
|
||||||
reduce_bytes += covered_part->getBytesOnDisk();
|
reduce_bytes += covered_part->getBytesOnDisk();
|
||||||
reduce_rows += covered_part->rows_count;
|
reduce_rows += covered_part->rows_count;
|
||||||
++reduce_parts;
|
++reduce_parts;
|
||||||
@ -2361,7 +2361,7 @@ bool MergeTreeData::renameTempPartAndReplace(
|
|||||||
decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
||||||
|
|
||||||
modifyPartState(part_it, DataPartState::Committed);
|
modifyPartState(part_it, DataPartState::Committed);
|
||||||
addPartContributionToColumnSizes(part);
|
addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||||
addPartContributionToDataVolume(part);
|
addPartContributionToDataVolume(part);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2404,7 +2404,7 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect
|
|||||||
{
|
{
|
||||||
if (part->getState() == IMergeTreeDataPart::State::Committed)
|
if (part->getState() == IMergeTreeDataPart::State::Committed)
|
||||||
{
|
{
|
||||||
removePartContributionToColumnSizes(part);
|
removePartContributionToColumnAndSecondaryIndexSizes(part);
|
||||||
removePartContributionToDataVolume(part);
|
removePartContributionToDataVolume(part);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2542,7 +2542,7 @@ restore_covered)
|
|||||||
if (part->getState() == DataPartState::Committed)
|
if (part->getState() == DataPartState::Committed)
|
||||||
{
|
{
|
||||||
removePartContributionToDataVolume(part);
|
removePartContributionToDataVolume(part);
|
||||||
removePartContributionToColumnSizes(part);
|
removePartContributionToColumnAndSecondaryIndexSizes(part);
|
||||||
}
|
}
|
||||||
modifyPartState(it_part, DataPartState::Deleting);
|
modifyPartState(it_part, DataPartState::Deleting);
|
||||||
|
|
||||||
@ -2590,7 +2590,7 @@ restore_covered)
|
|||||||
|
|
||||||
if ((*it)->getState() != DataPartState::Committed)
|
if ((*it)->getState() != DataPartState::Committed)
|
||||||
{
|
{
|
||||||
addPartContributionToColumnSizes(*it);
|
addPartContributionToColumnAndSecondaryIndexSizes(*it);
|
||||||
addPartContributionToDataVolume(*it);
|
addPartContributionToDataVolume(*it);
|
||||||
modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here
|
modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here
|
||||||
}
|
}
|
||||||
@ -2621,7 +2621,7 @@ restore_covered)
|
|||||||
|
|
||||||
if ((*it)->getState() != DataPartState::Committed)
|
if ((*it)->getState() != DataPartState::Committed)
|
||||||
{
|
{
|
||||||
addPartContributionToColumnSizes(*it);
|
addPartContributionToColumnAndSecondaryIndexSizes(*it);
|
||||||
addPartContributionToDataVolume(*it);
|
addPartContributionToDataVolume(*it);
|
||||||
modifyPartState(it, DataPartState::Committed);
|
modifyPartState(it, DataPartState::Committed);
|
||||||
}
|
}
|
||||||
@ -2973,17 +2973,17 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
|
|||||||
part->modification_time = disk->getLastModified(full_part_path).epochTime();
|
part->modification_time = disk->getLastModified(full_part_path).epochTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::calculateColumnSizesImpl()
|
void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl()
|
||||||
{
|
{
|
||||||
column_sizes.clear();
|
column_sizes.clear();
|
||||||
|
|
||||||
/// Take into account only committed parts
|
/// Take into account only committed parts
|
||||||
auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
|
auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
|
||||||
for (const auto & part : committed_parts_range)
|
for (const auto & part : committed_parts_range)
|
||||||
addPartContributionToColumnSizes(part);
|
addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::addPartContributionToColumnSizes(const DataPartPtr & part)
|
void MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part)
|
||||||
{
|
{
|
||||||
for (const auto & column : part->getColumns())
|
for (const auto & column : part->getColumns())
|
||||||
{
|
{
|
||||||
@ -2991,9 +2991,17 @@ void MergeTreeData::addPartContributionToColumnSizes(const DataPartPtr & part)
|
|||||||
ColumnSize part_column_size = part->getColumnSize(column.name, *column.type);
|
ColumnSize part_column_size = part->getColumnSize(column.name, *column.type);
|
||||||
total_column_size.add(part_column_size);
|
total_column_size.add(part_column_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
auto indexes_descriptions = getInMemoryMetadataPtr()->secondary_indices;
|
||||||
|
for (const auto & index : indexes_descriptions)
|
||||||
|
{
|
||||||
|
IndexSize & total_secondary_index_size = secondary_index_sizes[index.name];
|
||||||
|
IndexSize part_index_size = part->getSecondaryIndexSize(index.name);
|
||||||
|
total_secondary_index_size.add(part_index_size);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part)
|
void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part)
|
||||||
{
|
{
|
||||||
for (const auto & column : part->getColumns())
|
for (const auto & column : part->getColumns())
|
||||||
{
|
{
|
||||||
@ -3013,6 +3021,26 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
|
|||||||
log_subtract(total_column_size.data_uncompressed, part_column_size.data_uncompressed, ".data_uncompressed");
|
log_subtract(total_column_size.data_uncompressed, part_column_size.data_uncompressed, ".data_uncompressed");
|
||||||
log_subtract(total_column_size.marks, part_column_size.marks, ".marks");
|
log_subtract(total_column_size.marks, part_column_size.marks, ".marks");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
auto indexes_descriptions = getInMemoryMetadataPtr()->secondary_indices;
|
||||||
|
for (const auto & index : indexes_descriptions)
|
||||||
|
{
|
||||||
|
IndexSize & total_secondary_index_size = secondary_index_sizes[index.name];
|
||||||
|
IndexSize part_secondary_index_size = part->getSecondaryIndexSize(index.name);
|
||||||
|
|
||||||
|
auto log_subtract = [&](size_t & from, size_t value, const char * field)
|
||||||
|
{
|
||||||
|
if (value > from)
|
||||||
|
LOG_ERROR(log, "Possibly incorrect index size subtraction: {} - {} = {}, index: {}, field: {}",
|
||||||
|
from, value, from - value, index.name, field);
|
||||||
|
|
||||||
|
from -= value;
|
||||||
|
};
|
||||||
|
|
||||||
|
log_subtract(total_secondary_index_size.data_compressed, part_secondary_index_size.data_compressed, ".data_compressed");
|
||||||
|
log_subtract(total_secondary_index_size.data_uncompressed, part_secondary_index_size.data_uncompressed, ".data_uncompressed");
|
||||||
|
log_subtract(total_secondary_index_size.marks, part_secondary_index_size.marks, ".marks");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::checkAlterPartitionIsPossible(
|
void MergeTreeData::checkAlterPartitionIsPossible(
|
||||||
@ -4043,7 +4071,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
|||||||
reduce_rows += covered_part->rows_count;
|
reduce_rows += covered_part->rows_count;
|
||||||
|
|
||||||
data.modifyPartState(covered_part, DataPartState::Outdated);
|
data.modifyPartState(covered_part, DataPartState::Outdated);
|
||||||
data.removePartContributionToColumnSizes(covered_part);
|
data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part);
|
||||||
}
|
}
|
||||||
reduce_parts += covered_parts.size();
|
reduce_parts += covered_parts.size();
|
||||||
|
|
||||||
@ -4052,7 +4080,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
|||||||
++add_parts;
|
++add_parts;
|
||||||
|
|
||||||
data.modifyPartState(part, DataPartState::Committed);
|
data.modifyPartState(part, DataPartState::Committed);
|
||||||
data.addPartContributionToColumnSizes(part);
|
data.addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
||||||
|
@ -654,6 +654,12 @@ public:
|
|||||||
return column_sizes;
|
return column_sizes;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
IndexSizeByName getSecondaryIndexSizes() const override
|
||||||
|
{
|
||||||
|
auto lock = lockParts();
|
||||||
|
return secondary_index_sizes;
|
||||||
|
}
|
||||||
|
|
||||||
/// For ATTACH/DETACH/DROP PARTITION.
|
/// For ATTACH/DETACH/DROP PARTITION.
|
||||||
String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const;
|
String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const;
|
||||||
std::unordered_set<String> getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const;
|
std::unordered_set<String> getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const;
|
||||||
@ -873,6 +879,9 @@ protected:
|
|||||||
/// Current column sizes in compressed and uncompressed form.
|
/// Current column sizes in compressed and uncompressed form.
|
||||||
ColumnSizeByName column_sizes;
|
ColumnSizeByName column_sizes;
|
||||||
|
|
||||||
|
/// Current secondary index sizes in compressed and uncompressed form.
|
||||||
|
IndexSizeByName secondary_index_sizes;
|
||||||
|
|
||||||
/// Engine-specific methods
|
/// Engine-specific methods
|
||||||
BrokenPartCallback broken_part_callback;
|
BrokenPartCallback broken_part_callback;
|
||||||
|
|
||||||
@ -1005,11 +1014,12 @@ protected:
|
|||||||
|
|
||||||
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const;
|
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const;
|
||||||
|
|
||||||
/// Calculates column sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked.
|
/// Calculates column and secondary indexes sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked.
|
||||||
void calculateColumnSizesImpl();
|
void calculateColumnAndSecondaryIndexSizesImpl();
|
||||||
/// Adds or subtracts the contribution of the part to compressed column sizes.
|
|
||||||
void addPartContributionToColumnSizes(const DataPartPtr & part);
|
/// Adds or subtracts the contribution of the part to compressed column and secondary indexes sizes.
|
||||||
void removePartContributionToColumnSizes(const DataPartPtr & part);
|
void addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part);
|
||||||
|
void removePartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part);
|
||||||
|
|
||||||
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
|
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
|
||||||
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const;
|
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const;
|
||||||
|
@ -87,7 +87,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
|||||||
new_part->checksums = checksums;
|
new_part->checksums = checksums;
|
||||||
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
|
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
|
||||||
new_part->index_granularity = writer->getIndexGranularity();
|
new_part->index_granularity = writer->getIndexGranularity();
|
||||||
new_part->calculateColumnsSizesOnDisk();
|
new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||||
|
|
||||||
if (default_codec != nullptr)
|
if (default_codec != nullptr)
|
||||||
new_part->default_codec = default_codec;
|
new_part->default_codec = default_codec;
|
||||||
new_part->storage.lockSharedData(*new_part);
|
new_part->storage.lockSharedData(*new_part);
|
||||||
|
@ -475,7 +475,7 @@ void finalizeMutatedPart(
|
|||||||
new_data_part->setBytesOnDisk(
|
new_data_part->setBytesOnDisk(
|
||||||
MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath()));
|
MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath()));
|
||||||
new_data_part->default_codec = codec;
|
new_data_part->default_codec = codec;
|
||||||
new_data_part->calculateColumnsSizesOnDisk();
|
new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||||
new_data_part->storage.lockSharedData(*new_data_part);
|
new_data_part->storage.lockSharedData(*new_data_part);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -25,6 +25,9 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage
|
|||||||
{ "type", std::make_shared<DataTypeString>() },
|
{ "type", std::make_shared<DataTypeString>() },
|
||||||
{ "expr", std::make_shared<DataTypeString>() },
|
{ "expr", std::make_shared<DataTypeString>() },
|
||||||
{ "granularity", std::make_shared<DataTypeUInt64>() },
|
{ "granularity", std::make_shared<DataTypeUInt64>() },
|
||||||
|
{ "data_compressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||||
|
{ "data_uncompressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||||
|
{ "marks", std::make_shared<DataTypeUInt64>()}
|
||||||
}));
|
}));
|
||||||
setInMemoryMetadata(storage_metadata);
|
setInMemoryMetadata(storage_metadata);
|
||||||
}
|
}
|
||||||
@ -97,6 +100,7 @@ protected:
|
|||||||
continue;
|
continue;
|
||||||
const auto indices = metadata_snapshot->getSecondaryIndices();
|
const auto indices = metadata_snapshot->getSecondaryIndices();
|
||||||
|
|
||||||
|
auto secondary_index_sizes = table->getSecondaryIndexSizes();
|
||||||
for (const auto & index : indices)
|
for (const auto & index : indices)
|
||||||
{
|
{
|
||||||
++rows_count;
|
++rows_count;
|
||||||
@ -127,6 +131,21 @@ protected:
|
|||||||
// 'granularity' column
|
// 'granularity' column
|
||||||
if (column_mask[src_index++])
|
if (column_mask[src_index++])
|
||||||
res_columns[res_index++]->insert(index.granularity);
|
res_columns[res_index++]->insert(index.granularity);
|
||||||
|
|
||||||
|
auto & secondary_index_size = secondary_index_sizes[index.name];
|
||||||
|
|
||||||
|
// 'compressed bytes' column
|
||||||
|
if (column_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(secondary_index_size.data_compressed);
|
||||||
|
|
||||||
|
// 'uncompressed bytes' column
|
||||||
|
|
||||||
|
if (column_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(secondary_index_size.data_uncompressed);
|
||||||
|
|
||||||
|
/// 'marks' column
|
||||||
|
if (column_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(secondary_index_size.marks);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -30,6 +30,9 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_)
|
|||||||
{"data_compressed_bytes", std::make_shared<DataTypeUInt64>()},
|
{"data_compressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
{"data_uncompressed_bytes", std::make_shared<DataTypeUInt64>()},
|
{"data_uncompressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
{"marks_bytes", std::make_shared<DataTypeUInt64>()},
|
{"marks_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
|
{"secondary_indices_compressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
|
{"secondary_indices_uncompressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
|
{"secondary_indices_marks_bytes", std::make_shared<DataTypeUInt64>()},
|
||||||
{"modification_time", std::make_shared<DataTypeDateTime>()},
|
{"modification_time", std::make_shared<DataTypeDateTime>()},
|
||||||
{"remove_time", std::make_shared<DataTypeDateTime>()},
|
{"remove_time", std::make_shared<DataTypeDateTime>()},
|
||||||
{"refcount", std::make_shared<DataTypeUInt32>()},
|
{"refcount", std::make_shared<DataTypeUInt32>()},
|
||||||
@ -98,6 +101,7 @@ void StorageSystemParts::processNextStorage(
|
|||||||
auto part_state = all_parts_state[part_number];
|
auto part_state = all_parts_state[part_number];
|
||||||
|
|
||||||
ColumnSize columns_size = part->getTotalColumnsSize();
|
ColumnSize columns_size = part->getTotalColumnsSize();
|
||||||
|
ColumnSize secondary_indexes_size = part->getTotalSeconaryIndicesSize();
|
||||||
|
|
||||||
size_t src_index = 0, res_index = 0;
|
size_t src_index = 0, res_index = 0;
|
||||||
if (columns_mask[src_index++])
|
if (columns_mask[src_index++])
|
||||||
@ -126,6 +130,12 @@ void StorageSystemParts::processNextStorage(
|
|||||||
columns[res_index++]->insert(columns_size.data_uncompressed);
|
columns[res_index++]->insert(columns_size.data_uncompressed);
|
||||||
if (columns_mask[src_index++])
|
if (columns_mask[src_index++])
|
||||||
columns[res_index++]->insert(columns_size.marks);
|
columns[res_index++]->insert(columns_size.marks);
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
columns[res_index++]->insert(secondary_indexes_size.data_compressed);
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
columns[res_index++]->insert(secondary_indexes_size.data_uncompressed);
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
columns[res_index++]->insert(secondary_indexes_size.marks);
|
||||||
if (columns_mask[src_index++])
|
if (columns_mask[src_index++])
|
||||||
columns[res_index++]->insert(static_cast<UInt64>(part->modification_time));
|
columns[res_index++]->insert(static_cast<UInt64>(part->modification_time));
|
||||||
|
|
||||||
|
@ -1,8 +1,8 @@
|
|||||||
default data_01917 d1_idx minmax d1 1
|
default data_01917 d1_idx minmax d1 1 0 0 0
|
||||||
default data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1
|
default data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 0 0 0
|
||||||
default data_01917_2 memory set frequency * length(name) 5
|
default data_01917_2 memory set frequency * length(name) 5 0 0 0
|
||||||
default data_01917_2 sample_index1 minmax length(name), name 4
|
default data_01917_2 sample_index1 minmax length(name), name 4 0 0 0
|
||||||
default data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4
|
default data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 0 0 0
|
||||||
2
|
2
|
||||||
3
|
3
|
||||||
d1_idx
|
d1_idx
|
||||||
|
@ -1,9 +1,9 @@
|
|||||||
default alter_index_test index_a set a 1
|
default alter_index_test index_a set a 1 0 0 0
|
||||||
default alter_index_test index_b minmax b 1
|
default alter_index_test index_b minmax b 1 0 0 0
|
||||||
default alter_index_test index_c set c 2
|
default alter_index_test index_c set c 2 0 0 0
|
||||||
default alter_index_test index_a set a 1
|
default alter_index_test index_a set a 1 0 0 0
|
||||||
default alter_index_test index_d set d 1
|
default alter_index_test index_d set d 1 0 0 0
|
||||||
default alter_index_test index_b minmax b 1
|
default alter_index_test index_b minmax b 1 0 0 0
|
||||||
default alter_index_test index_c set c 2
|
default alter_index_test index_c set c 2 0 0 0
|
||||||
default alter_index_test index_a set a 1
|
default alter_index_test index_a set a 1 0 0 0
|
||||||
default alter_index_test index_d set d 1
|
default alter_index_test index_d set d 1 0 0 0
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
default test_table value_index minmax value 1 38 12 24
|
@ -0,0 +1,15 @@
|
|||||||
|
DROP TABLE IF EXISTS test_table;
|
||||||
|
|
||||||
|
CREATE TABLE test_table
|
||||||
|
(
|
||||||
|
key UInt64,
|
||||||
|
value String,
|
||||||
|
INDEX value_index value TYPE minmax GRANULARITY 1
|
||||||
|
)
|
||||||
|
Engine=MergeTree()
|
||||||
|
ORDER BY key;
|
||||||
|
|
||||||
|
INSERT INTO test_table VALUES (0, 'Value');
|
||||||
|
SELECT * FROM system.data_skipping_indices WHERE database = currentDatabase();
|
||||||
|
|
||||||
|
DROP TABLE test_table;
|
Loading…
Reference in New Issue
Block a user