mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
System table data skipping indices added size
This commit is contained in:
parent
c368b20dce
commit
2d069acc22
@ -10,6 +10,9 @@ Columns:
|
||||
- `type` ([String](../../sql-reference/data-types/string.md)) — Index type.
|
||||
- `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.
|
||||
- `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**
|
||||
|
||||
@ -26,6 +29,9 @@ name: clicks_idx
|
||||
type: minmax
|
||||
expr: clicks
|
||||
granularity: 1
|
||||
data_compressed_bytes: 58
|
||||
data_uncompressed_bytes: 6
|
||||
marks: 48
|
||||
|
||||
Row 2:
|
||||
──────
|
||||
@ -35,4 +41,7 @@ name: contacts_null_idx
|
||||
type: minmax
|
||||
expr: assumeNotNull(contacts_null)
|
||||
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.
|
||||
|
||||
- `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.
|
||||
|
||||
- `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
|
||||
data_compressed_bytes: 157
|
||||
data_uncompressed_bytes: 91
|
||||
secondary_indices_compressed_bytes: 58
|
||||
secondary_indices_uncompressed_bytes: 6
|
||||
secondary_indices_marks_bytes: 48
|
||||
marks_bytes: 144
|
||||
modification_time: 2020-06-18 13:01:49
|
||||
remove_time: 1970-01-01 00:00:00
|
||||
|
@ -87,6 +87,8 @@ struct ColumnSize
|
||||
}
|
||||
};
|
||||
|
||||
using IndexSize = ColumnSize;
|
||||
|
||||
/** Storage. Describes the table. Responsible for
|
||||
* - storage of the table data;
|
||||
* - 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>;
|
||||
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
|
||||
/// multiversion, so it can be concurrently changed, but returned copy can be
|
||||
/// used without any locks.
|
||||
|
@ -584,7 +584,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
|
||||
loadColumns(require_columns_checksums);
|
||||
loadChecksums(require_columns_checksums);
|
||||
loadIndexGranularity();
|
||||
calculateColumnsSizesOnDisk();
|
||||
calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||
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`.
|
||||
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);
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk()
|
||||
{
|
||||
calculateColumnsSizesOnDisk();
|
||||
calculateSecondaryIndicesSizesOnDisk();
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
|
||||
{
|
||||
@ -1429,6 +1434,40 @@ void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
|
||||
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;
|
||||
|
||||
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
|
||||
{
|
||||
/// For some types of parts columns_size maybe not calculated
|
||||
@ -1439,6 +1478,15 @@ ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const I
|
||||
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
|
||||
{
|
||||
for (const auto & [column_name, size] : columns_sizes)
|
||||
|
@ -55,6 +55,8 @@ public:
|
||||
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||
using NameToNumber = std::unordered_map<std::string, size_t>;
|
||||
|
||||
using IndexSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||
|
||||
using Type = MergeTreeDataPartType;
|
||||
|
||||
|
||||
@ -101,9 +103,16 @@ public:
|
||||
/// Otherwise return information about column size on disk.
|
||||
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
|
||||
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 ~IMergeTreeDataPart();
|
||||
@ -341,7 +350,9 @@ public:
|
||||
|
||||
/// Calculate the total size of the entire directory with all the files
|
||||
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;
|
||||
|
||||
@ -396,6 +407,10 @@ protected:
|
||||
/// Size for each column, calculated once in calcuateColumnSizesOnDisk
|
||||
ColumnSizeByName columns_sizes;
|
||||
|
||||
ColumnSize total_secondary_indices_size;
|
||||
|
||||
IndexSizeByName secondary_index_sizes;
|
||||
|
||||
/// Total size on disk, not only columns. May not contain size of
|
||||
/// checksums.txt and columns.txt. 0 - if not counted;
|
||||
UInt64 bytes_on_disk{0};
|
||||
@ -450,6 +465,10 @@ private:
|
||||
|
||||
void loadPartitionAndMinMaxIndex();
|
||||
|
||||
void calculateColumnsSizesOnDisk();
|
||||
|
||||
void calculateSecondaryIndicesSizesOnDisk();
|
||||
|
||||
/// 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.
|
||||
|
@ -1167,7 +1167,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
}
|
||||
}
|
||||
|
||||
calculateColumnSizesImpl();
|
||||
calculateColumnAndSecondaryIndexSizesImpl();
|
||||
|
||||
|
||||
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);
|
||||
modifyPartState(covered_part, DataPartState::Outdated);
|
||||
removePartContributionToColumnSizes(covered_part);
|
||||
removePartContributionToColumnAndSecondaryIndexSizes(covered_part);
|
||||
reduce_bytes += covered_part->getBytesOnDisk();
|
||||
reduce_rows += covered_part->rows_count;
|
||||
++reduce_parts;
|
||||
@ -2361,7 +2361,7 @@ bool MergeTreeData::renameTempPartAndReplace(
|
||||
decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
||||
|
||||
modifyPartState(part_it, DataPartState::Committed);
|
||||
addPartContributionToColumnSizes(part);
|
||||
addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
addPartContributionToDataVolume(part);
|
||||
}
|
||||
|
||||
@ -2404,7 +2404,7 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect
|
||||
{
|
||||
if (part->getState() == IMergeTreeDataPart::State::Committed)
|
||||
{
|
||||
removePartContributionToColumnSizes(part);
|
||||
removePartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
removePartContributionToDataVolume(part);
|
||||
}
|
||||
|
||||
@ -2542,7 +2542,7 @@ restore_covered)
|
||||
if (part->getState() == DataPartState::Committed)
|
||||
{
|
||||
removePartContributionToDataVolume(part);
|
||||
removePartContributionToColumnSizes(part);
|
||||
removePartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
}
|
||||
modifyPartState(it_part, DataPartState::Deleting);
|
||||
|
||||
@ -2590,7 +2590,7 @@ restore_covered)
|
||||
|
||||
if ((*it)->getState() != DataPartState::Committed)
|
||||
{
|
||||
addPartContributionToColumnSizes(*it);
|
||||
addPartContributionToColumnAndSecondaryIndexSizes(*it);
|
||||
addPartContributionToDataVolume(*it);
|
||||
modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here
|
||||
}
|
||||
@ -2621,7 +2621,7 @@ restore_covered)
|
||||
|
||||
if ((*it)->getState() != DataPartState::Committed)
|
||||
{
|
||||
addPartContributionToColumnSizes(*it);
|
||||
addPartContributionToColumnAndSecondaryIndexSizes(*it);
|
||||
addPartContributionToDataVolume(*it);
|
||||
modifyPartState(it, DataPartState::Committed);
|
||||
}
|
||||
@ -2973,32 +2973,46 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
|
||||
part->modification_time = disk->getLastModified(full_part_path).epochTime();
|
||||
}
|
||||
|
||||
void MergeTreeData::calculateColumnSizesImpl()
|
||||
void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl()
|
||||
{
|
||||
std::cerr << "MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl" << std::endl;
|
||||
|
||||
column_sizes.clear();
|
||||
|
||||
/// Take into account only committed parts
|
||||
auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
|
||||
for (const auto & part : committed_parts_range)
|
||||
addPartContributionToColumnSizes(part);
|
||||
addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
}
|
||||
|
||||
void MergeTreeData::addPartContributionToColumnSizes(const DataPartPtr & part)
|
||||
void MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part)
|
||||
{
|
||||
std::cerr << "MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes " << part->name << std::endl;
|
||||
|
||||
for (const auto & column : part->getColumns())
|
||||
{
|
||||
std::cerr << "Column name " << column.name << std::endl;
|
||||
ColumnSize & total_column_size = column_sizes[column.name];
|
||||
std::cerr << "Total column size compressed " << total_column_size.data_compressed << " uncompressed size " << total_column_size.data_uncompressed << std::endl;
|
||||
ColumnSize part_column_size = part->getColumnSize(column.name, *column.type);
|
||||
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())
|
||||
{
|
||||
ColumnSize & total_column_size = column_sizes[column.name];
|
||||
ColumnSize part_column_size = part->getColumnSize(column.name, *column.type);
|
||||
ColumnSize part_secondary_index_size = part->getColumnSize(column.name, *column.type);
|
||||
|
||||
auto log_subtract = [&](size_t & from, size_t value, const char * field)
|
||||
{
|
||||
@ -3009,9 +3023,29 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
|
||||
from -= value;
|
||||
};
|
||||
|
||||
log_subtract(total_column_size.data_compressed, part_column_size.data_compressed, ".data_compressed");
|
||||
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.data_compressed, part_secondary_index_size.data_compressed, ".data_compressed");
|
||||
log_subtract(total_column_size.data_uncompressed, part_secondary_index_size.data_uncompressed, ".data_uncompressed");
|
||||
log_subtract(total_column_size.marks, part_secondary_index_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");
|
||||
}
|
||||
}
|
||||
|
||||
@ -4043,7 +4077,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
||||
reduce_rows += covered_part->rows_count;
|
||||
|
||||
data.modifyPartState(covered_part, DataPartState::Outdated);
|
||||
data.removePartContributionToColumnSizes(covered_part);
|
||||
data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part);
|
||||
}
|
||||
reduce_parts += covered_parts.size();
|
||||
|
||||
@ -4052,7 +4086,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
||||
++add_parts;
|
||||
|
||||
data.modifyPartState(part, DataPartState::Committed);
|
||||
data.addPartContributionToColumnSizes(part);
|
||||
data.addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
}
|
||||
}
|
||||
data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
|
||||
|
@ -654,6 +654,12 @@ public:
|
||||
return column_sizes;
|
||||
}
|
||||
|
||||
IndexSizeByName getSecondaryIndexSizes() const override
|
||||
{
|
||||
auto lock = lockParts();
|
||||
return secondary_index_sizes;
|
||||
}
|
||||
|
||||
/// For ATTACH/DETACH/DROP PARTITION.
|
||||
String getPartitionIDFromQuery(const ASTPtr & ast, 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.
|
||||
ColumnSizeByName column_sizes;
|
||||
|
||||
/// Current secondary index sizes in compressed and uncompressed form.
|
||||
IndexSizeByName secondary_index_sizes;
|
||||
|
||||
/// Engine-specific methods
|
||||
BrokenPartCallback broken_part_callback;
|
||||
|
||||
@ -1005,11 +1014,12 @@ protected:
|
||||
|
||||
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.
|
||||
void calculateColumnSizesImpl();
|
||||
/// Adds or subtracts the contribution of the part to compressed column sizes.
|
||||
void addPartContributionToColumnSizes(const DataPartPtr & part);
|
||||
void removePartContributionToColumnSizes(const DataPartPtr & part);
|
||||
/// Calculates column and secondary indexes sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked.
|
||||
void calculateColumnAndSecondaryIndexSizesImpl();
|
||||
|
||||
/// Adds or subtracts the contribution of the part to compressed column and secondary indexes sizes.
|
||||
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.
|
||||
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const;
|
||||
|
@ -87,7 +87,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
||||
new_part->checksums = checksums;
|
||||
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
|
||||
new_part->index_granularity = writer->getIndexGranularity();
|
||||
new_part->calculateColumnsSizesOnDisk();
|
||||
new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||
|
||||
if (default_codec != nullptr)
|
||||
new_part->default_codec = default_codec;
|
||||
new_part->storage.lockSharedData(*new_part);
|
||||
|
@ -475,7 +475,7 @@ void finalizeMutatedPart(
|
||||
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();
|
||||
new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||
new_data_part->storage.lockSharedData(*new_data_part);
|
||||
}
|
||||
|
||||
|
@ -25,6 +25,9 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage
|
||||
{ "type", std::make_shared<DataTypeString>() },
|
||||
{ "expr", std::make_shared<DataTypeString>() },
|
||||
{ "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);
|
||||
}
|
||||
@ -97,6 +100,7 @@ protected:
|
||||
continue;
|
||||
const auto indices = metadata_snapshot->getSecondaryIndices();
|
||||
|
||||
auto secondary_index_sizes = table->getSecondaryIndexSizes();
|
||||
for (const auto & index : indices)
|
||||
{
|
||||
++rows_count;
|
||||
@ -127,6 +131,21 @@ protected:
|
||||
// 'granularity' column
|
||||
if (column_mask[src_index++])
|
||||
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_uncompressed_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>()},
|
||||
{"remove_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"refcount", std::make_shared<DataTypeUInt32>()},
|
||||
@ -98,6 +101,7 @@ void StorageSystemParts::processNextStorage(
|
||||
auto part_state = all_parts_state[part_number];
|
||||
|
||||
ColumnSize columns_size = part->getTotalColumnsSize();
|
||||
ColumnSize secondary_indexes_size = part->getTotalSeconaryIndicesSize();
|
||||
|
||||
size_t src_index = 0, res_index = 0;
|
||||
if (columns_mask[src_index++])
|
||||
@ -126,6 +130,12 @@ void StorageSystemParts::processNextStorage(
|
||||
columns[res_index++]->insert(columns_size.data_uncompressed);
|
||||
if (columns_mask[src_index++])
|
||||
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++])
|
||||
columns[res_index++]->insert(static_cast<UInt64>(part->modification_time));
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user