mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-01 03:52:15 +00:00
polymorphic parts (development)
This commit is contained in:
parent
18163e4d7f
commit
8ba37da2ef
@ -354,6 +354,7 @@ void SystemLog<LogElement>::flushImpl(EntryType reason)
|
||||
/// In case of exception, also clean accumulated data - to avoid locking.
|
||||
data.clear();
|
||||
}
|
||||
|
||||
if (reason == EntryType::FORCE_FLUSH)
|
||||
{
|
||||
std::lock_guard lock(condvar_mutex);
|
||||
|
@ -137,27 +137,33 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other)
|
||||
IMergeTreeDataPart::IMergeTreeDataPart(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: storage(storage_)
|
||||
, name(name_)
|
||||
, info(MergeTreePartInfo::fromPartName(name_, storage.format_version))
|
||||
, index_granularity_info(index_granularity_info_)
|
||||
, disk(disk_)
|
||||
, relative_path(relative_path_.value_or(name_))
|
||||
, index_granularity_info(shared_from_this()) {}
|
||||
{
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::IMergeTreeDataPart(
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: storage(storage_)
|
||||
, name(name_)
|
||||
, info(info_)
|
||||
, index_granularity_info(index_granularity_info_)
|
||||
, disk(disk_)
|
||||
, relative_path(relative_path_.value_or(name_))
|
||||
, index_granularity_info(shared_from_this()) {}
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & type) const
|
||||
|
@ -74,7 +74,7 @@ public:
|
||||
/// NOTE: Returns zeros if column files are not found in checksums.
|
||||
/// NOTE: You must ensure that no ALTERs are in progress when calculating ColumnSizes.
|
||||
/// (either by locking columns_lock, or by locking table structure).
|
||||
virtual ColumnSize getColumnSize(const String & name, const IDataType & type) const = 0;
|
||||
ColumnSize getColumnSize(const String & name, const IDataType & type);
|
||||
|
||||
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
||||
/// Load checksums from checksums.txt if exists. Load index if required.
|
||||
@ -84,10 +84,6 @@ public:
|
||||
/// If no checksums are present returns the name of the first physically existing column.
|
||||
virtual String getColumnNameWithMinumumCompressedSize() const = 0;
|
||||
|
||||
virtual String getMarkExtension(bool /* is_adaptive */) const { return ""; }
|
||||
|
||||
virtual size_t getMarkSize(bool /* is_adaptive */) const { return 0; }
|
||||
|
||||
// virtual void detach() = 0;
|
||||
|
||||
// virtual Checksums check(
|
||||
@ -110,7 +106,7 @@ public:
|
||||
enum class Type
|
||||
{
|
||||
WIDE,
|
||||
STRIPED,
|
||||
COMPACT,
|
||||
IN_MEMORY,
|
||||
};
|
||||
|
||||
@ -124,7 +120,7 @@ public:
|
||||
{
|
||||
case Type::WIDE:
|
||||
return "Wide";
|
||||
case Type::STRIPED:
|
||||
case Type::COMPACT:
|
||||
return "Striped";
|
||||
case Type::IN_MEMORY:
|
||||
return "InMemory";
|
||||
@ -141,17 +137,21 @@ public:
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk = {},
|
||||
const std::optional<String> & relative_path = {});
|
||||
|
||||
IMergeTreeDataPart(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk = {},
|
||||
const std::optional<String> & relative_path = {});
|
||||
|
||||
void assertOnDisk() const;
|
||||
|
||||
ColumnSize getColumnSize(const String & column_name, const IDataType & type) const;
|
||||
|
||||
ColumnSize getTotalColumnsSize() const;
|
||||
|
||||
/// Generate the new name for this part according to `new_part_info` and min/max dates from the old name.
|
||||
@ -174,6 +174,7 @@ public:
|
||||
|
||||
String name;
|
||||
MergeTreePartInfo info;
|
||||
MergeTreeIndexGranularityInfo index_granularity_info;
|
||||
|
||||
DiskSpace::DiskPtr disk;
|
||||
|
||||
@ -317,8 +318,6 @@ public:
|
||||
*/
|
||||
mutable std::mutex alter_mutex;
|
||||
|
||||
MergeTreeIndexGranularityInfo index_granularity_info;
|
||||
|
||||
/// For data in RAM ('index')
|
||||
UInt64 getIndexSizeInBytes() const;
|
||||
UInt64 getIndexSizeInAllocatedBytes() const;
|
||||
|
95
dbms/src/Storages/MergeTree/IMergeTreeDataPartWriter.h
Normal file
95
dbms/src/Storages/MergeTree/IMergeTreeDataPartWriter.h
Normal file
@ -0,0 +1,95 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeDataPartWriter
|
||||
{
|
||||
public:
|
||||
using WrittenOffsetColumns = std::set<std::string>;
|
||||
|
||||
struct ColumnStream
|
||||
{
|
||||
ColumnStream(
|
||||
const String & escaped_column_name_,
|
||||
const String & data_path_,
|
||||
const std::string & data_file_extension_,
|
||||
const std::string & marks_path_,
|
||||
const std::string & marks_file_extension_,
|
||||
const CompressionCodecPtr & compression_codec_,
|
||||
size_t max_compress_block_size_,
|
||||
size_t estimated_size_,
|
||||
size_t aio_threshold_);
|
||||
|
||||
String escaped_column_name;
|
||||
std::string data_file_extension;
|
||||
std::string marks_file_extension;
|
||||
|
||||
/// compressed -> compressed_buf -> plain_hashing -> plain_file
|
||||
std::unique_ptr<WriteBufferFromFileBase> plain_file;
|
||||
HashingWriteBuffer plain_hashing;
|
||||
CompressedWriteBuffer compressed_buf;
|
||||
HashingWriteBuffer compressed;
|
||||
|
||||
/// marks -> marks_file
|
||||
WriteBufferFromFile marks_file;
|
||||
HashingWriteBuffer marks;
|
||||
|
||||
void finalize();
|
||||
|
||||
void sync();
|
||||
|
||||
void addToChecksums(MergeTreeData::DataPart::Checksums & checksums);
|
||||
};
|
||||
|
||||
virtual size_t write(
|
||||
const Block & block, size_t from_mark, size_t offset,
|
||||
/* Blocks with already sorted index columns */
|
||||
const Block & primary_key_block = {}, const Block & skip_indexes_block = {}) = 0;
|
||||
|
||||
virtual std::pair<size_t, size_t> writeColumn(
|
||||
const String & name,
|
||||
const IDataType & type,
|
||||
const IColumn & column,
|
||||
WrittenOffsetColumns & offset_columns,
|
||||
bool skip_offsets,
|
||||
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
size_t from_mark) = 0;
|
||||
|
||||
// /// Write single granule of one column (rows between 2 marks)
|
||||
// virtual size_t writeSingleGranule(
|
||||
// const String & name,
|
||||
// const IDataType & type,
|
||||
// const IColumn & column,
|
||||
// WrittenOffsetColumns & offset_columns,
|
||||
// bool skip_offsets,
|
||||
// IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
// IDataType::SerializeBinaryBulkSettings & serialize_settings,
|
||||
// size_t from_row,
|
||||
// size_t number_of_rows,
|
||||
// bool write_marks) = 0;
|
||||
|
||||
// /// Write mark for column
|
||||
// virtual void writeSingleMark(
|
||||
// const String & name,
|
||||
// const IDataType & type,
|
||||
// WrittenOffsetColumns & offset_columns,
|
||||
// bool skip_offsets,
|
||||
// size_t number_of_rows,
|
||||
// DB::IDataType::SubstreamPath & path) = 0;
|
||||
protected:
|
||||
void start();
|
||||
|
||||
const NamesAndTypesList & columns_list;
|
||||
IColumn::Permutation * permutation;
|
||||
bool started = false;
|
||||
};
|
||||
|
||||
}
|
@ -288,6 +288,13 @@ std::pair<size_t, size_t> IMergedBlockOutputStream::writeColumn(
|
||||
}
|
||||
}, serialize_settings.path);
|
||||
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) name: " << name << "\n";
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) from_mark: " << from_mark << "\n";
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) current_column_mark: " << current_column_mark << "\n";
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) current_row: " << current_row << "\n";
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) total_rows: " << total_rows;
|
||||
std::cerr << "(IMergedBlockOutputStream::writeColumn) blocks_are_granules_size: " << blocks_are_granules_size << "\n";
|
||||
|
||||
return std::make_pair(current_column_mark, current_row - total_rows);
|
||||
}
|
||||
|
||||
|
@ -149,6 +149,8 @@ Block MergeTreeBaseSelectBlockInputStream::readFromPartImpl()
|
||||
UInt64 recommended_rows = estimateNumRows(*task, task->range_reader);
|
||||
UInt64 rows_to_read = std::max(UInt64(1), std::min(current_max_block_size_rows, recommended_rows));
|
||||
|
||||
std::cerr << "(readFromPartImpl) rows_to_read: " << rows_to_read << "\n";
|
||||
|
||||
auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges);
|
||||
|
||||
/// All rows were filtered. Repeat.
|
||||
|
@ -283,12 +283,16 @@ void MergeTreeData::setProperties(
|
||||
|
||||
Names new_primary_key_columns;
|
||||
Names new_sorting_key_columns;
|
||||
NameSet sorting_key_columns_set;
|
||||
|
||||
for (size_t i = 0; i < sorting_key_size; ++i)
|
||||
{
|
||||
String sorting_key_column = new_sorting_key_expr_list->children[i]->getColumnName();
|
||||
new_sorting_key_columns.push_back(sorting_key_column);
|
||||
|
||||
if (!sorting_key_columns_set.emplace(sorting_key_column).second)
|
||||
throw Exception("Sorting key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (i < primary_key_size)
|
||||
{
|
||||
String pk_column = new_primary_key_expr_list->children[i]->getColumnName();
|
||||
|
@ -746,6 +746,7 @@ protected:
|
||||
|
||||
friend class IMergeTreeDataPart;
|
||||
friend class MergeTreeDataPartWide;
|
||||
friend class MergeTreeDataPartCompact;
|
||||
friend class MergeTreeDataMergerMutator;
|
||||
friend class ReplicatedMergeTreeAlterThread;
|
||||
friend struct ReplicatedMergeTreeTableMetadata;
|
||||
|
@ -1170,6 +1170,10 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer
|
||||
if (need_remove_expired_values)
|
||||
return MergeAlgorithm::Horizontal;
|
||||
|
||||
for (const auto & part : parts)
|
||||
if (!part->supportsVerticalMerge())
|
||||
return MergeAlgorithm::Horizontal;
|
||||
|
||||
bool is_supported_storage =
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
||||
|
@ -0,0 +1,687 @@
|
||||
#include "MergeTreeDataPartCompact.h"
|
||||
|
||||
#include <optional>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Compression/CompressionInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <common/JSON.h>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// namespace
|
||||
// {
|
||||
// }
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int NO_FILE_IN_DATA_PART;
|
||||
extern const int EXPECTED_END_OF_FILE;
|
||||
extern const int CORRUPTED_DATA;
|
||||
extern const int NOT_FOUND_EXPECTED_DATA_PART;
|
||||
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
|
||||
extern const int BAD_TTL_FILE;
|
||||
extern const int CANNOT_UNLINK;
|
||||
}
|
||||
|
||||
|
||||
static ReadBufferFromFile openForReading(const String & path)
|
||||
{
|
||||
return ReadBufferFromFile(path, std::min(static_cast<Poco::File::FileSize>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
|
||||
}
|
||||
|
||||
MergeTreeDataPartCompact::MergeTreeDataPartCompact(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, index_granularity_info_, disk_, relative_path_)
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeDataPartCompact::MergeTreeDataPartCompact(
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, index_granularity_info_, disk_, relative_path_)
|
||||
{
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
const NamesAndTypesList & columns_to_read,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const ReaderSettings & reader_settings,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const
|
||||
{
|
||||
return std::make_unique<MergeTreeReaderCompact>(shared_from_this(), columns_to_read, uncompressed_cache,
|
||||
mark_cache, mark_ranges, reader_settings, avg_value_size_hints);
|
||||
}
|
||||
|
||||
|
||||
/// Takes into account the fact that several columns can e.g. share their .size substreams.
|
||||
/// When calculating totals these should be counted only once.
|
||||
ColumnSize MergeTreeDataPartCompact::getColumnSizeImpl(
|
||||
const String & column_name, const IDataType & type, std::unordered_set<String> * processed_substreams) const
|
||||
{
|
||||
UNUSED(column_name);
|
||||
UNUSED(type);
|
||||
UNUSED(processed_substreams);
|
||||
// ColumnSize size;
|
||||
// if (checksums.empty())
|
||||
// return size;
|
||||
|
||||
// type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||
// {
|
||||
// String file_name = IDataType::getFileNameForStream(column_name, substream_path);
|
||||
|
||||
// if (processed_substreams && !processed_substreams->insert(file_name).second)
|
||||
// return;
|
||||
|
||||
// auto bin_checksum = checksums.files.find(file_name + ".bin");
|
||||
// if (bin_checksum != checksums.files.end())
|
||||
// {
|
||||
// size.data_compressed += bin_checksum->second.file_size;
|
||||
// size.data_uncompressed += bin_checksum->second.uncompressed_size;
|
||||
// }
|
||||
|
||||
// auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension);
|
||||
// if (mrk_checksum != checksums.files.end())
|
||||
// size.marks += mrk_checksum->second.file_size;
|
||||
// }, {});
|
||||
|
||||
return ColumnSize{};
|
||||
}
|
||||
|
||||
/** Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
||||
* If no checksums are present returns the name of the first physically existing column.
|
||||
*/
|
||||
String MergeTreeDataPartCompact::getColumnNameWithMinumumCompressedSize() const
|
||||
{
|
||||
const auto & storage_columns = storage.getColumns().getAllPhysical();
|
||||
const std::string * minimum_size_column = nullptr;
|
||||
UInt64 minimum_size = std::numeric_limits<UInt64>::max();
|
||||
|
||||
for (const auto & column : storage_columns)
|
||||
{
|
||||
if (!hasColumnFiles(column.name, *column.type))
|
||||
continue;
|
||||
|
||||
const auto size = getColumnSize(column.name, *column.type).data_compressed;
|
||||
if (size < minimum_size)
|
||||
{
|
||||
minimum_size = size;
|
||||
minimum_size_column = &column.name;
|
||||
}
|
||||
}
|
||||
|
||||
if (!minimum_size_column)
|
||||
throw Exception("Could not find a column of minimum size in MergeTree, part " + getFullPath(), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return *minimum_size_column;
|
||||
}
|
||||
|
||||
MergeTreeDataPartCompact::~MergeTreeDataPartCompact()
|
||||
{
|
||||
if (state == State::DeleteOnDestroy || is_temp)
|
||||
{
|
||||
try
|
||||
{
|
||||
std::string path = getFullPath();
|
||||
|
||||
Poco::File dir(path);
|
||||
if (!dir.exists())
|
||||
return;
|
||||
|
||||
if (is_temp)
|
||||
{
|
||||
String file_name = Poco::Path(relative_path).getFileName();
|
||||
|
||||
if (file_name.empty())
|
||||
throw Exception("relative_path " + relative_path + " of part " + name + " is invalid or not set", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!startsWith(file_name, "tmp"))
|
||||
{
|
||||
LOG_ERROR(storage.log, "~DataPart() should remove part " << path
|
||||
<< " but its name doesn't start with tmp. Too suspicious, keeping the part.");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
dir.remove(true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 MergeTreeDataPartCompact::calculateTotalSizeOnDisk(const String & from)
|
||||
{
|
||||
Poco::File cur(from);
|
||||
if (cur.isFile())
|
||||
return cur.getSize();
|
||||
std::vector<std::string> files;
|
||||
cur.list(files);
|
||||
UInt64 res = 0;
|
||||
for (const auto & file : files)
|
||||
res += calculateTotalSizeOnDisk(from + file);
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::remove() const
|
||||
{
|
||||
if (relative_path.empty())
|
||||
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/** Atomic directory removal:
|
||||
* - rename directory to temporary name;
|
||||
* - remove it recursive.
|
||||
*
|
||||
* For temporary name we use "delete_tmp_" prefix.
|
||||
*
|
||||
* NOTE: We cannot use "tmp_delete_" prefix, because there is a second thread,
|
||||
* that calls "clearOldTemporaryDirectories" and removes all directories, that begin with "tmp_" and are old enough.
|
||||
* But when we removing data part, it can be old enough. And rename doesn't change mtime.
|
||||
* And a race condition can happen that will lead to "File not found" error here.
|
||||
*/
|
||||
|
||||
String full_path = storage.getFullPathOnDisk(disk);
|
||||
String from = full_path + relative_path;
|
||||
String to = full_path + "delete_tmp_" + name;
|
||||
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
|
||||
|
||||
|
||||
Poco::File from_dir{from};
|
||||
Poco::File to_dir{to};
|
||||
|
||||
if (to_dir.exists())
|
||||
{
|
||||
LOG_WARNING(storage.log, "Directory " << to << " (to which part must be renamed before removing) already exists."
|
||||
" Most likely this is due to unclean restart. Removing it.");
|
||||
|
||||
try
|
||||
{
|
||||
to_dir.remove(true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(storage.log, "Cannot remove directory " << to << ". Check owner and access rights.");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
from_dir.renameTo(to);
|
||||
}
|
||||
catch (const Poco::FileNotFoundException &)
|
||||
{
|
||||
LOG_ERROR(storage.log, "Directory " << from << " (part to remove) doesn't exist or one of nested files has gone."
|
||||
" Most likely this is due to manual removing. This should be discouraged. Ignoring.");
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
/// Remove each expected file in directory, then remove directory itself.
|
||||
|
||||
#if !__clang__
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-variable"
|
||||
#endif
|
||||
std::shared_lock<std::shared_mutex> lock(columns_lock);
|
||||
|
||||
for (const auto & [file, _] : checksums.files)
|
||||
{
|
||||
String path_to_remove = to + "/" + file;
|
||||
if (0 != unlink(path_to_remove.c_str()))
|
||||
throwFromErrnoWithPath("Cannot unlink file " + path_to_remove, path_to_remove,
|
||||
ErrorCodes::CANNOT_UNLINK);
|
||||
}
|
||||
#if !__clang__
|
||||
#pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
for (const auto & file : {"checksums.txt", "columns.txt"})
|
||||
{
|
||||
String path_to_remove = to + "/" + file;
|
||||
if (0 != unlink(path_to_remove.c_str()))
|
||||
throwFromErrnoWithPath("Cannot unlink file " + path_to_remove, path_to_remove,
|
||||
ErrorCodes::CANNOT_UNLINK);
|
||||
}
|
||||
|
||||
if (0 != rmdir(to.c_str()))
|
||||
throwFromErrnoWithPath("Cannot rmdir file " + to, to, ErrorCodes::CANNOT_UNLINK);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
|
||||
|
||||
LOG_ERROR(storage.log, "Cannot quickly remove directory " << to << " by removing files; fallback to recursive removal. Reason: "
|
||||
<< getCurrentExceptionMessage(false));
|
||||
|
||||
to_dir.remove(true);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool /* check_consistency */)
|
||||
{
|
||||
/// Memory should not be limited during ATTACH TABLE query.
|
||||
/// This is already true at the server startup but must be also ensured for manual table ATTACH.
|
||||
/// Motivation: memory for index is shared between queries - not belong to the query itself.
|
||||
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
|
||||
|
||||
loadColumns(require_columns_checksums);
|
||||
loadChecksums(require_columns_checksums);
|
||||
loadIndexGranularity();
|
||||
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
|
||||
loadRowsCount(); /// Must be called after loadIndex() as it uses the value of `index_granularity`.
|
||||
loadPartitionAndMinMaxIndex();
|
||||
loadTTLInfos();
|
||||
// if (check_consistency)
|
||||
// checkConsistency(require_columns_checksums);
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadIndexGranularity()
|
||||
{
|
||||
String full_path = getFullPath();
|
||||
|
||||
if (columns.empty())
|
||||
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
/// We can use any column, it doesn't matter
|
||||
std::string marks_file_path = index_granularity_info.getMarksFilePath(full_path + "data");
|
||||
if (!Poco::File(marks_file_path).exists())
|
||||
throw Exception("Marks file '" + marks_file_path + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
size_t marks_file_size = Poco::File(marks_file_path).getSize();
|
||||
|
||||
/// old version of marks with static index granularity
|
||||
ReadBufferFromFile buffer(marks_file_path, marks_file_size, -1);
|
||||
while (!buffer.eof())
|
||||
{
|
||||
size_t granularity;
|
||||
readIntBinary(granularity, buffer);
|
||||
index_granularity.appendMark(granularity);
|
||||
/// Skip offsets for columns
|
||||
buffer.seek(sizeof(size_t) * 2 * columns.size());
|
||||
}
|
||||
if (index_granularity.getMarksCount() * index_granularity_info.mark_size_in_bytes != marks_file_size)
|
||||
throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadIndex()
|
||||
{
|
||||
/// It can be empty in case of mutations
|
||||
if (!index_granularity.isInitialized())
|
||||
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t key_size = storage.primary_key_columns.size();
|
||||
|
||||
if (key_size)
|
||||
{
|
||||
MutableColumns loaded_index;
|
||||
loaded_index.resize(key_size);
|
||||
|
||||
for (size_t i = 0; i < key_size; ++i)
|
||||
{
|
||||
loaded_index[i] = storage.primary_key_data_types[i]->createColumn();
|
||||
loaded_index[i]->reserve(index_granularity.getMarksCount());
|
||||
}
|
||||
|
||||
String index_path = getFullPath() + "primary.idx";
|
||||
ReadBufferFromFile index_file = openForReading(index_path);
|
||||
|
||||
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) //-V756
|
||||
for (size_t j = 0; j < key_size; ++j)
|
||||
storage.primary_key_data_types[j]->deserializeBinary(*loaded_index[j], index_file);
|
||||
|
||||
for (size_t i = 0; i < key_size; ++i)
|
||||
{
|
||||
loaded_index[i]->protect();
|
||||
if (loaded_index[i]->size() != index_granularity.getMarksCount())
|
||||
throw Exception("Cannot read all data from index file " + index_path
|
||||
+ "(expected size: " + toString(index_granularity.getMarksCount()) + ", read: " + toString(loaded_index[i]->size()) + ")",
|
||||
ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
}
|
||||
|
||||
if (!index_file.eof())
|
||||
throw Exception("Index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
|
||||
|
||||
index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end()));
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadPartitionAndMinMaxIndex()
|
||||
{
|
||||
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
DayNum min_date;
|
||||
DayNum max_date;
|
||||
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
|
||||
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date));
|
||||
minmax_idx = MinMaxIndex(min_date, max_date);
|
||||
}
|
||||
else
|
||||
{
|
||||
String path = getFullPath();
|
||||
partition.load(storage, path);
|
||||
if (!isEmpty())
|
||||
minmax_idx.load(storage, path);
|
||||
}
|
||||
|
||||
String calculated_partition_id = partition.getID(storage.partition_key_sample);
|
||||
if (calculated_partition_id != info.partition_id)
|
||||
throw Exception(
|
||||
"While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id
|
||||
+ " differs from partition ID in part name: " + info.partition_id,
|
||||
ErrorCodes::CORRUPTED_DATA);
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadChecksums(bool require)
|
||||
{
|
||||
String path = getFullPath() + "checksums.txt";
|
||||
Poco::File checksums_file(path);
|
||||
if (checksums_file.exists())
|
||||
{
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
if (checksums.read(file))
|
||||
{
|
||||
assertEOF(file);
|
||||
bytes_on_disk = checksums.getTotalSizeOnDisk();
|
||||
}
|
||||
else
|
||||
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (require)
|
||||
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadRowsCount()
|
||||
{
|
||||
if (index_granularity.empty())
|
||||
{
|
||||
rows_count = 0;
|
||||
}
|
||||
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
String path = getFullPath() + "count.txt";
|
||||
if (!Poco::File(path).exists())
|
||||
throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
readIntText(rows_count, file);
|
||||
assertEOF(file);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const NameAndTypePair & column : columns)
|
||||
{
|
||||
ColumnPtr column_col = column.type->createColumn();
|
||||
if (!column_col->isFixedAndContiguous() || column_col->lowCardinality())
|
||||
continue;
|
||||
|
||||
size_t column_size = getColumnSize(column.name, *column.type).data_uncompressed;
|
||||
if (!column_size)
|
||||
continue;
|
||||
|
||||
size_t sizeof_field = column_col->sizeOfValueIfFixed();
|
||||
rows_count = column_size / sizeof_field;
|
||||
|
||||
if (column_size % sizeof_field != 0)
|
||||
{
|
||||
throw Exception(
|
||||
"Uncompressed size of column " + column.name + "(" + toString(column_size)
|
||||
+ ") is not divisible by the size of value (" + toString(sizeof_field) + ")",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
size_t last_mark_index_granularity = index_granularity.getLastNonFinalMarkRows();
|
||||
size_t rows_approx = index_granularity.getTotalRows();
|
||||
if (!(rows_count <= rows_approx && rows_approx < rows_count + last_mark_index_granularity))
|
||||
throw Exception(
|
||||
"Unexpected size of column " + column.name + ": " + toString(rows_count) + " rows, expected "
|
||||
+ toString(rows_approx) + "+-" + toString(last_mark_index_granularity) + " rows according to the index",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
throw Exception("Data part doesn't contain fixed size column (even Date column)", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::loadTTLInfos()
|
||||
{
|
||||
String path = getFullPath() + "ttl.txt";
|
||||
if (Poco::File(path).exists())
|
||||
{
|
||||
ReadBufferFromFile in = openForReading(path);
|
||||
assertString("ttl format version: ", in);
|
||||
size_t format_version;
|
||||
readText(format_version, in);
|
||||
assertChar('\n', in);
|
||||
|
||||
if (format_version == 1)
|
||||
{
|
||||
try
|
||||
{
|
||||
ttl_infos.read(in);
|
||||
}
|
||||
catch (const JSONException &)
|
||||
{
|
||||
throw Exception("Error while parsing file ttl.txt in part: " + name, ErrorCodes::BAD_TTL_FILE);
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown ttl format version: " + toString(format_version), ErrorCodes::BAD_TTL_FILE);
|
||||
}
|
||||
}
|
||||
|
||||
// void MergeTreeDataPartCompact::accumulateColumnSizes(ColumnToSize & column_to_size) const
|
||||
// {
|
||||
// std::shared_lock<std::shared_mutex> part_lock(columns_lock);
|
||||
|
||||
// for (const NameAndTypePair & name_type : storage.getColumns().getAllPhysical())
|
||||
// {
|
||||
// IDataType::SubstreamPath path;
|
||||
// name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||
// {
|
||||
// Poco::File bin_file(getFullPath() + IDataType::getFileNameForStream(name_type.name, substream_path) + ".bin");
|
||||
// if (bin_file.exists())
|
||||
// column_to_size[name_type.name] += bin_file.getSize();
|
||||
// }, path);
|
||||
// }
|
||||
// }
|
||||
|
||||
void MergeTreeDataPartCompact::loadColumns(bool require)
|
||||
{
|
||||
String path = getFullPath() + "columns.txt";
|
||||
Poco::File poco_file_path{path};
|
||||
if (!poco_file_path.exists())
|
||||
{
|
||||
if (require)
|
||||
throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
/// If there is no file with a list of columns, write it down.
|
||||
for (const NameAndTypePair & column : storage.getColumns().getAllPhysical())
|
||||
if (Poco::File(getFullPath() + escapeForFileName(column.name) + ".bin").exists())
|
||||
columns.push_back(column);
|
||||
|
||||
if (columns.empty())
|
||||
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
{
|
||||
WriteBufferFromFile out(path + ".tmp", 4096);
|
||||
columns.writeText(out);
|
||||
}
|
||||
Poco::File(path + ".tmp").renameTo(path);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
is_frozen = !poco_file_path.canWrite();
|
||||
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
columns.readText(file);
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata)
|
||||
{
|
||||
String path = getFullPath();
|
||||
|
||||
if (!checksums.empty())
|
||||
{
|
||||
if (!storage.primary_key_columns.empty() && !checksums.files.count("primary.idx"))
|
||||
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (require_part_metadata)
|
||||
{
|
||||
for (const NameAndTypePair & name_type : columns)
|
||||
{
|
||||
IDataType::SubstreamPath stream_path;
|
||||
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||
{
|
||||
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
||||
String mrk_file_name = file_name + index_granularity_info.marks_file_extension;
|
||||
String bin_file_name = file_name + ".bin";
|
||||
if (!checksums.files.count(mrk_file_name))
|
||||
throw Exception("No " + mrk_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
if (!checksums.files.count(bin_file_name))
|
||||
throw Exception("No " + bin_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
}, stream_path);
|
||||
}
|
||||
}
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
if (!checksums.files.count("count.txt"))
|
||||
throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (storage.partition_key_expr && !checksums.files.count("partition.dat"))
|
||||
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (!isEmpty())
|
||||
{
|
||||
for (const String & col_name : storage.minmax_idx_columns)
|
||||
{
|
||||
if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx"))
|
||||
throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
checksums.checkSizes(path);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto check_file_not_empty = [&path](const String & file_path)
|
||||
{
|
||||
Poco::File file(file_path);
|
||||
if (!file.exists() || file.getSize() == 0)
|
||||
throw Exception("Part " + path + " is broken: " + file_path + " is empty", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
||||
return file.getSize();
|
||||
};
|
||||
|
||||
/// Check that the primary key index is not empty.
|
||||
if (!storage.primary_key_columns.empty())
|
||||
check_file_not_empty(path + "primary.idx");
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
check_file_not_empty(path + "count.txt");
|
||||
|
||||
if (storage.partition_key_expr)
|
||||
check_file_not_empty(path + "partition.dat");
|
||||
|
||||
for (const String & col_name : storage.minmax_idx_columns)
|
||||
check_file_not_empty(path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
||||
}
|
||||
|
||||
/// Check that all marks are nonempty and have the same size.
|
||||
|
||||
std::optional<UInt64> marks_size;
|
||||
for (const NameAndTypePair & name_type : columns)
|
||||
{
|
||||
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||
{
|
||||
Poco::File file(IDataType::getFileNameForStream(name_type.name, substream_path) + index_granularity_info.marks_file_extension);
|
||||
|
||||
/// Missing file is Ok for case when new column was added.
|
||||
if (file.exists())
|
||||
{
|
||||
UInt64 file_size = file.getSize();
|
||||
|
||||
if (!file_size)
|
||||
throw Exception("Part " + path + " is broken: " + file.path() + " is empty.",
|
||||
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
||||
|
||||
if (!marks_size)
|
||||
marks_size = file_size;
|
||||
else if (file_size != *marks_size)
|
||||
throw Exception("Part " + path + " is broken: marks have different sizes.",
|
||||
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// bool MergeTreeDataPartCompact::hasColumnFiles(const String & column_name, const IDataType & type) const
|
||||
// {
|
||||
// bool res = true;
|
||||
|
||||
// type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||
// {
|
||||
// String file_name = IDataType::getFileNameForStream(column_name, substream_path);
|
||||
|
||||
// auto bin_checksum = checksums.files.find(file_name + ".bin");
|
||||
// auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension);
|
||||
|
||||
// if (bin_checksum == checksums.files.end() || mrk_checksum == checksums.files.end())
|
||||
// res = false;
|
||||
// }, {});
|
||||
|
||||
// return res;
|
||||
// }
|
||||
|
||||
}
|
@ -34,18 +34,20 @@ public:
|
||||
using Checksums = MergeTreeDataPartChecksums;
|
||||
using Checksum = MergeTreeDataPartChecksums::Checksum;
|
||||
|
||||
MergeTreeDataPartCompact(
|
||||
MergeTreeDataPartCompact(
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const DiskSpace::DiskPtr & disk,
|
||||
const std::optional<String> & relative_path = {});
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
|
||||
MergeTreeDataPartCompact(
|
||||
MergeTreeDataPartCompact(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const DiskSpace::DiskPtr & disk,
|
||||
const std::optional<String> & relative_path = {});
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
@ -58,19 +60,8 @@ public:
|
||||
|
||||
bool isStoredOnDisk() const override { return true; }
|
||||
|
||||
String getMarkExtension(bool /* is_adaptive */) const override { return ".mrk3"; }
|
||||
|
||||
bool getMarkSize(bool is_adaptive)
|
||||
{
|
||||
return sizeof(size_t) + columns.size() * sizeof(size_t) * 2;
|
||||
}
|
||||
|
||||
void remove() const override;
|
||||
|
||||
/// NOTE: Returns zeros if column files are not found in checksums.
|
||||
/// NOTE: You must ensure that no ALTERs are in progress when calculating ColumnSizes.
|
||||
/// (either by locking columns_lock, or by locking table structure).
|
||||
ColumnSize getColumnSize(const String & name, const IDataType & type) const override;
|
||||
|
||||
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
||||
/// Load checksums from checksums.txt if exists. Load index if required.
|
||||
@ -80,7 +71,7 @@ public:
|
||||
/// If no checksums are present returns the name of the first physically existing column.
|
||||
String getColumnNameWithMinumumCompressedSize() const override;
|
||||
|
||||
virtual Type getType() const override { return Type::WIDE; }
|
||||
virtual Type getType() const override { return Type::COMPACT; }
|
||||
|
||||
~MergeTreeDataPartCompact() override;
|
||||
|
||||
|
@ -1,16 +1,19 @@
|
||||
#include "MergeTreeDataPartFactory.h"
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
std::shared_ptr<IMergeTreeDataPart> createPart(const MergeTreeData & storage, const DiskSpace::DiskPtr & disk, const String & name,
|
||||
const MergeTreePartInfo & info, const String & relative_path)
|
||||
{
|
||||
return std::make_shared<MergeTreeDataPartWide>(storage, name, info, disk, relative_path);
|
||||
MergeTreeIndexGranularityInfo index_granularity_info(storage, ".mrk2", 3 * sizeof(size_t));
|
||||
return std::make_shared<MergeTreeDataPartWide>(storage, name, info, index_granularity_info, disk, relative_path);
|
||||
}
|
||||
|
||||
std::shared_ptr<IMergeTreeDataPart> createPart(MergeTreeData & storage, const DiskSpace::DiskPtr & disk,
|
||||
const String & name, const String & relative_path)
|
||||
{
|
||||
return std::make_shared<MergeTreeDataPartWide>(storage, name, disk, relative_path);
|
||||
MergeTreeIndexGranularityInfo index_granularity_info(storage, ".mrk2", 3 * sizeof(size_t));
|
||||
return std::make_shared<MergeTreeDataPartWide>(storage, name, index_granularity_info, disk, relative_path);
|
||||
}
|
||||
}
|
||||
|
@ -54,9 +54,10 @@ static ReadBufferFromFile openForReading(const String & path)
|
||||
MergeTreeDataPartWide::MergeTreeDataPartWide(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, disk_, relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, index_granularity_info_, disk_, relative_path_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -64,9 +65,10 @@ MergeTreeDataPartWide::MergeTreeDataPartWide(
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, disk_, relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, index_granularity_info_, disk_, relative_path_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -115,11 +117,6 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl(
|
||||
return size;
|
||||
}
|
||||
|
||||
ColumnSize MergeTreeDataPartWide::getColumnSize(const String & column_name, const IDataType & type) const
|
||||
{
|
||||
return getColumnSizeImpl(column_name, type, nullptr);
|
||||
}
|
||||
|
||||
/** Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
||||
* If no checksums are present returns the name of the first physically existing column.
|
||||
*/
|
||||
@ -134,7 +131,7 @@ String MergeTreeDataPartWide::getColumnNameWithMinumumCompressedSize() const
|
||||
if (!hasColumnFiles(column.name, *column.type))
|
||||
continue;
|
||||
|
||||
const auto size = getColumnSize(column.name, *column.type).data_compressed;
|
||||
const auto size = getColumnSizeImpl(column.name, *column.type, nullptr).data_compressed;
|
||||
if (size < minimum_size)
|
||||
{
|
||||
minimum_size = size;
|
||||
@ -315,7 +312,7 @@ void MergeTreeDataPartWide::loadColumnsChecksumsIndexes(bool require_columns_che
|
||||
void MergeTreeDataPartWide::loadIndexGranularity()
|
||||
{
|
||||
String full_path = getFullPath();
|
||||
index_granularity_info.changeGranularityIfRequired(shared_from_this());
|
||||
index_granularity_info.changeGranularityIfRequired(full_path);
|
||||
|
||||
if (columns.empty())
|
||||
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
@ -327,7 +324,6 @@ void MergeTreeDataPartWide::loadIndexGranularity()
|
||||
|
||||
size_t marks_file_size = Poco::File(marks_file_path).getSize();
|
||||
|
||||
/// old version of marks with static index granularity
|
||||
if (!index_granularity_info.is_adaptive)
|
||||
{
|
||||
size_t marks_count = marks_file_size / index_granularity_info.mark_size_in_bytes;
|
||||
@ -467,7 +463,7 @@ void MergeTreeDataPartWide::loadRowsCount()
|
||||
if (!column_col->isFixedAndContiguous() || column_col->lowCardinality())
|
||||
continue;
|
||||
|
||||
size_t column_size = getColumnSize(column.name, *column.type).data_uncompressed;
|
||||
size_t column_size = getColumnSizeImpl(column.name, *column.type, nullptr).data_uncompressed;
|
||||
if (!column_size)
|
||||
continue;
|
||||
|
||||
|
@ -38,12 +38,14 @@ public:
|
||||
const MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk,
|
||||
const std::optional<String> & relative_path = {});
|
||||
|
||||
MergeTreeDataPartWide(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const DiskSpace::DiskPtr & disk,
|
||||
const std::optional<String> & relative_path = {});
|
||||
|
||||
@ -62,19 +64,6 @@ public:
|
||||
|
||||
bool supportsVerticalMerge() const override { return true; }
|
||||
|
||||
String getMarkExtension(bool is_adaptive) const override { return is_adaptive ? ".mrk2" : ".mrk"; }
|
||||
|
||||
size_t getMarkSize(bool is_adaptive) const override
|
||||
{
|
||||
size_t nums = is_adaptive ? 3 : 2;
|
||||
return sizeof(size_t) * nums;
|
||||
}
|
||||
|
||||
/// NOTE: Returns zeros if column files are not found in checksums.
|
||||
/// NOTE: You must ensure that no ALTERs are in progress when calculating ColumnSizes.
|
||||
/// (either by locking columns_lock, or by locking table structure).
|
||||
ColumnSize getColumnSize(const String & name, const IDataType & type) const override;
|
||||
|
||||
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
||||
/// Load checksums from checksums.txt if exists. Load index if required.
|
||||
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) override;
|
||||
@ -83,7 +72,7 @@ public:
|
||||
/// If no checksums are present returns the name of the first physically existing column.
|
||||
String getColumnNameWithMinumumCompressedSize() const override;
|
||||
|
||||
virtual Type getType() const override { return Type::WIDE; }
|
||||
Type getType() const override { return Type::WIDE; }
|
||||
|
||||
~MergeTreeDataPartWide() override;
|
||||
|
||||
|
116
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp
Normal file
116
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp
Normal file
@ -0,0 +1,116 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
size_t MergeTreeDataPartWriterCompact::writeColumnSingleGranule(
|
||||
const ColumnWithTypeAndName & column,
|
||||
WrittenOffsetColumns & offset_columns,
|
||||
bool skip_offsets,
|
||||
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
IDataType::SerializeBinaryBulkSettings & serialize_settings,
|
||||
size_t from_row,
|
||||
size_t number_of_rows)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeDataPartWriterCompact::write(const Block & block, size_t from_mark, size_t offset,
|
||||
const Block & primary_key_block, const Block & skip_indexes_block)
|
||||
{
|
||||
if (!started)
|
||||
start();
|
||||
|
||||
size_t total_rows = block.rows();
|
||||
size_t current_mark = from_mark;
|
||||
size_t current_row = 0;
|
||||
|
||||
|
||||
IDataType::SerializeBinaryBulkSettings serialize_settings;
|
||||
serialize_settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; };
|
||||
serialize_settings.position_independent_encoding = false;
|
||||
serialize_settings.low_cardinality_max_dictionary_size = 0;
|
||||
|
||||
ColumnsWithTypeAndName columns_to_write(columns_list.size());
|
||||
auto it = columns_list.begin();
|
||||
for (size_t i = 0; i < columns_list.size(); ++i, ++it)
|
||||
{
|
||||
if (permutation)
|
||||
{
|
||||
if (primary_key_block.has(it->name))
|
||||
columns_to_write[i] = primary_key_block.getByName(it->name);
|
||||
else if (skip_indexes_block.has(it->name))
|
||||
columns_to_write[i] = skip_indexes_block.getByName(it->name);
|
||||
else
|
||||
{
|
||||
columns_to_write[i] = block.getByName(it->name);
|
||||
columns_to_write[i].column = columns_to_write[i].column->permute(*permutation, 0);
|
||||
}
|
||||
}
|
||||
else
|
||||
columns_to_write[i] = block.getByName(it->name);
|
||||
}
|
||||
|
||||
while (current_row < total_rows)
|
||||
{
|
||||
bool write_marks = true;
|
||||
size_t rows_to_write;
|
||||
if (current_row == 0 && offset != 0)
|
||||
{
|
||||
rows_to_write = offset;
|
||||
write_marks = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
rows_to_write = index_granularity->getMarkRows(current_mark);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < columns_to_write.size(); ++i)
|
||||
{
|
||||
current_row = writeColumnSingleGranule(columns_to_write[i], offset_columns, skip_offsets, serialization_states[i], serialize_settings, current_row, rows_to_write);
|
||||
}
|
||||
|
||||
if (write_marks)
|
||||
{
|
||||
writeMark();
|
||||
++current_mark;
|
||||
}
|
||||
}
|
||||
|
||||
/// We always write end granule for block in Compact parts.
|
||||
return 0;
|
||||
}
|
||||
|
||||
size_t MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTypeAndName & column,
|
||||
WrittenOffsetColumns & offset_columns,
|
||||
bool skip_offsets,
|
||||
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
IDataType::SerializeBinaryBulkSettings & serialize_settings,
|
||||
size_t from_row,
|
||||
size_t number_of_rows)
|
||||
{
|
||||
column.type->serializeBinaryBulkStatePrefix(serialize_settings, serialization_state);
|
||||
column.type->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, serialization_state);
|
||||
column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_state);
|
||||
}
|
||||
|
||||
void MergeTreeDataPartWriterWide::start()
|
||||
{
|
||||
if (started)
|
||||
return;
|
||||
|
||||
started = true;
|
||||
|
||||
serialization_states.reserve(columns_list.size());
|
||||
WrittenOffsetColumns tmp_offset_columns;
|
||||
IDataType::SerializeBinaryBulkSettings settings;
|
||||
|
||||
for (const auto & col : columns_list)
|
||||
{
|
||||
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
||||
serialization_states.emplace_back(nullptr);
|
||||
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
42
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h
Normal file
42
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h
Normal file
@ -0,0 +1,42 @@
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeDataPartWriterCompact : IMergeTreeDataPartWriter
|
||||
{
|
||||
public:
|
||||
size_t write(const Block & block, size_t from_mark, size_t offset,
|
||||
const Block & primary_key_block, const Block & skip_indexes_block) override;
|
||||
|
||||
std::pair<size_t, size_t> writeColumn(
|
||||
const String & name,
|
||||
const IDataType & type,
|
||||
const IColumn & column,
|
||||
WrittenOffsetColumns & offset_columns,
|
||||
bool skip_offsets,
|
||||
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
size_t from_mark) override;
|
||||
|
||||
/// Write single granule of one column (rows between 2 marks)
|
||||
size_t writeColumnSingleGranule(
|
||||
const ColumnWithTypeAndName & column,
|
||||
WrittenOffsetColumns & offset_columns,
|
||||
bool skip_offsets,
|
||||
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
IDataType::SerializeBinaryBulkSettings & serialize_settings,
|
||||
size_t from_row,
|
||||
size_t number_of_rows);
|
||||
|
||||
void writeSingleMark()
|
||||
|
||||
protected:
|
||||
void start() override;
|
||||
|
||||
private:
|
||||
ColumnStream stream;
|
||||
MergeTreeIndexGranularity * index_granularity = nullptr;
|
||||
Columns columns_to_write;
|
||||
};
|
||||
|
||||
}
|
67
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp
Normal file
67
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp
Normal file
@ -0,0 +1,67 @@
|
||||
// #include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
|
||||
// size_t MergeTreeDataPartWriterWide::write(size_t current_mark, const Block & block)
|
||||
// {
|
||||
// if (!started)
|
||||
// start();
|
||||
|
||||
// size_t index_offset = 0;
|
||||
// auto it = columns_list.begin();
|
||||
// for (size_t i = 0; i < columns_list.size(); ++i, ++it)
|
||||
// {
|
||||
// const ColumnWithTypeAndName & column = block.getByName(it->name);
|
||||
|
||||
// if (permutation)
|
||||
// {
|
||||
// auto primary_column_it = primary_key_column_name_to_position.find(it->name);
|
||||
// auto skip_index_column_it = skip_indexes_column_name_to_position.find(it->name);
|
||||
|
||||
// if (primary_key_column_name_to_position.end() != primary_column_it)
|
||||
// {
|
||||
// const auto & primary_column = *primary_key_columns[primary_column_it->second].column;
|
||||
// std::tie(std::ignore, index_offset) = writeColumn(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i], current_mark);
|
||||
// }
|
||||
// else if (skip_indexes_column_name_to_position.end() != skip_index_column_it)
|
||||
// {
|
||||
// const auto & index_column = *skip_indexes_columns[skip_index_column_it->second].column;
|
||||
// std::tie(std::ignore, index_offset) = writeColumn(column.name, *column.type, index_column, offset_columns, false, serialization_states[i], current_mark);
|
||||
// }
|
||||
// else
|
||||
// {
|
||||
// /// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM.
|
||||
// ColumnPtr permuted_column = column.column->permute(*permutation, 0);
|
||||
// std::tie(std::ignore, index_offset) = writeColumn(column.name, *column.type, *permuted_column, offset_columns, false, serialization_states[i], current_mark);
|
||||
// }
|
||||
// }
|
||||
// else
|
||||
// {
|
||||
// std::tie(std::ignore, index_offset) = writeColumn(column.name, *column.type, *column.column, offset_columns, false, serialization_states[i], current_mark);
|
||||
// }
|
||||
// }
|
||||
|
||||
// return index_offset;
|
||||
// }
|
||||
|
||||
// void MergeTreeDataPartWriterWide::start()
|
||||
// {
|
||||
// if (started)
|
||||
// return;
|
||||
|
||||
// started = true;
|
||||
|
||||
// serialization_states.reserve(columns_list.size());
|
||||
// WrittenOffsetColumns tmp_offset_columns;
|
||||
// IDataType::SerializeBinaryBulkSettings settings;
|
||||
|
||||
// for (const auto & col : columns_list)
|
||||
// {
|
||||
// settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
||||
// serialization_states.emplace_back(nullptr);
|
||||
// col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
||||
// }
|
||||
// }
|
||||
|
||||
// }
|
28
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h
Normal file
28
dbms/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h
Normal file
@ -0,0 +1,28 @@
|
||||
// #include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
|
||||
// class MergeTreeDataPartWriterWide : IMergeTreeDataPartWriter
|
||||
// {
|
||||
// public:
|
||||
// size_t write(size_t current_mark, const Block & block) override;
|
||||
|
||||
// std::pair<size_t, size_t> writeColumn(
|
||||
// const String & name,
|
||||
// const IDataType & type,
|
||||
// const IColumn & column,
|
||||
// WrittenOffsetColumns & offset_columns,
|
||||
// bool skip_offsets,
|
||||
// IDataType::SerializeBinaryBulkStatePtr & serialization_state,
|
||||
// size_t from_mark) override;
|
||||
|
||||
// protected:
|
||||
// void start() override;
|
||||
|
||||
// private:
|
||||
// SerializationStates serialization_states;
|
||||
// NameSet permuted_columns;
|
||||
// };
|
||||
|
||||
// }
|
@ -168,13 +168,14 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition)
|
||||
{
|
||||
Block & block = block_with_partition.block;
|
||||
std::cerr << "(MergeTreeDataWriter::writeTempPart) block.rows(): " << block.rows() << "\n";
|
||||
|
||||
static const String TMP_PREFIX = "tmp_insert_";
|
||||
|
||||
/// This will generate unique name in scope of current server process.
|
||||
Int64 temp_index = data.insert_increment.get();
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex minmax_idx;
|
||||
IMergeTreeDataPart::MinMaxIndex minmax_idx;
|
||||
minmax_idx.update(block, data.minmax_idx_columns);
|
||||
|
||||
MergeTreePartition partition(std::move(block_with_partition.partition));
|
||||
|
@ -29,30 +29,25 @@ std::optional<std::string> MergeTreeIndexGranularityInfo::getMrkExtensionFromFS(
|
||||
return {};
|
||||
}
|
||||
|
||||
MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeDataPartPtr & part)
|
||||
MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeData & storage,
|
||||
const String & marks_file_extension_, UInt8 mark_size_in_bytes_)
|
||||
: marks_file_extension(marks_file_extension_), mark_size_in_bytes(mark_size_in_bytes_)
|
||||
{
|
||||
const auto storage_settings = part->storage.getSettings();
|
||||
const auto storage_settings = storage.getSettings();
|
||||
fixed_index_granularity = storage_settings->index_granularity;
|
||||
/// Granularity is fixed
|
||||
if (!part->storage.canUseAdaptiveGranularity())
|
||||
if (!storage.canUseAdaptiveGranularity())
|
||||
setNonAdaptive();
|
||||
else
|
||||
setAdaptive(storage_settings->index_granularity_bytes);
|
||||
|
||||
mark_size_in_bytes = part->getMarkSize(is_adaptive);
|
||||
marks_file_extension = part->getMarkExtension(is_adaptive);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeIndexGranularityInfo::changeGranularityIfRequired(const MergeTreeDataPartPtr & part)
|
||||
void MergeTreeIndexGranularityInfo::changeGranularityIfRequired(const String & path)
|
||||
{
|
||||
auto mrk_ext = getMrkExtensionFromFS(part->getFullPath());
|
||||
auto mrk_ext = getMrkExtensionFromFS(path);
|
||||
if (mrk_ext && *mrk_ext == ".mrk") /// TODO
|
||||
{
|
||||
setNonAdaptive();
|
||||
mark_size_in_bytes = part->getMarkSize(is_adaptive);
|
||||
marks_file_extension = part->getMarkExtension(is_adaptive);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranularityInfo::setAdaptive(size_t index_granularity_bytes_)
|
||||
|
@ -31,9 +31,10 @@ public:
|
||||
/// Approximate bytes size of one granule
|
||||
size_t index_granularity_bytes;
|
||||
|
||||
MergeTreeIndexGranularityInfo(const MergeTreeDataPartPtr & part);
|
||||
MergeTreeIndexGranularityInfo(const MergeTreeData & storage,
|
||||
const String & mark_file_extension_, UInt8 mark_size_in_bytes_);
|
||||
|
||||
void changeGranularityIfRequired(const MergeTreeDataPartPtr & part);
|
||||
void changeGranularityIfRequired(const String & path);
|
||||
|
||||
String getMarksFilePath(const String & path_prefix) const
|
||||
{
|
||||
|
@ -4,6 +4,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_FOUND_EXPECTED_DATA_PART;
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
MergeTreeReaderCompact::MergeTreeReaderCompact(const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_,
|
||||
const MarkRanges & mark_ranges_, const ReaderSettings & settings_, const ValueSizeMap & avg_value_size_hints_)
|
||||
@ -11,6 +19,30 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(const MergeTreeData::DataPartPtr
|
||||
, uncompressed_cache_, mark_cache_, mark_ranges_
|
||||
, settings_, avg_value_size_hints_)
|
||||
{
|
||||
size_t buffer_size = settings.max_read_buffer_size;
|
||||
|
||||
if (uncompressed_cache)
|
||||
{
|
||||
auto buffer = std::make_unique<CachedCompressedReadBuffer>(
|
||||
"data.bin", uncompressed_cache, 0, settings.min_bytes_to_use_direct_io, buffer_size);
|
||||
|
||||
// if (profile_callback)
|
||||
// buffer->setProfileCallback(profile_callback, clock_type);
|
||||
|
||||
cached_buffer = std::move(buffer);
|
||||
data_buffer = cached_buffer.get();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buffer = std::make_unique<CompressedReadBufferFromFile>(
|
||||
"data.bin", 0, settings.min_bytes_to_use_direct_io, buffer_size);
|
||||
|
||||
// if (profile_callback)
|
||||
// buffer->setProfileCallback(profile_callback, clock_type);
|
||||
|
||||
non_cached_buffer = std::move(buffer);
|
||||
data_buffer = non_cached_buffer.get();
|
||||
}
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res)
|
||||
@ -20,9 +52,68 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading,
|
||||
UNUSED(max_rows_to_read);
|
||||
UNUSED(res);
|
||||
|
||||
return 0;
|
||||
size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark);
|
||||
size_t read_rows = 0;
|
||||
|
||||
size_t ind = 0;
|
||||
for (const auto & it : columns)
|
||||
{
|
||||
bool append = res.has(it.name);
|
||||
if (!append)
|
||||
res.insert(ColumnWithTypeAndName(it.type->createColumn(), it.type, it.name));
|
||||
|
||||
/// To keep offsets shared. TODO Very dangerous. Get rid of this.
|
||||
MutableColumnPtr column = res.getByName(it.name).column->assumeMutable();
|
||||
|
||||
try
|
||||
{
|
||||
size_t column_size_before_reading = column->size();
|
||||
|
||||
readData(it.name, *it.type, *column, from_mark, ind++, rows_to_read);
|
||||
|
||||
/// For elements of Nested, column_size_before_reading may be greater than column size
|
||||
/// if offsets are not empty and were already read, but elements are empty.
|
||||
if (column->size())
|
||||
read_rows = std::max(read_rows, column->size() - column_size_before_reading);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
/// Better diagnostics.
|
||||
e.addMessage("(while reading column " + it.name + ")");
|
||||
throw;
|
||||
}
|
||||
|
||||
if (column->size())
|
||||
res.getByName(it.name).column = std::move(column);
|
||||
else
|
||||
res.erase(it.name);
|
||||
}
|
||||
|
||||
return read_rows;
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeReaderCompact::readData(
|
||||
const String & name, const IDataType & type, IColumn & column,
|
||||
size_t from_mark, size_t column_position, size_t rows_to_read)
|
||||
{
|
||||
seekToMark(from_mark, column_position);
|
||||
|
||||
IDataType::DeserializeBinaryBulkSettings deserialize_settings;
|
||||
deserialize_settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return data_buffer; };
|
||||
deserialize_settings.avg_value_size_hint = avg_value_size_hints[name];
|
||||
deserialize_settings.position_independent_encoding = false;
|
||||
|
||||
IDataType::DeserializeBinaryBulkStatePtr state;
|
||||
type.deserializeBinaryBulkStatePrefix(deserialize_settings, state);
|
||||
type.deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state);
|
||||
|
||||
if (column.size() != rows_to_read)
|
||||
throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column.size()) + ". Rows expected: "+ toString(rows_to_read) + ".",
|
||||
ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeReaderCompact::loadMarks()
|
||||
{
|
||||
const auto & index_granularity_info = data_part->index_granularity_info;
|
||||
@ -64,5 +155,42 @@ void MergeTreeReaderCompact::loadMarks()
|
||||
marks = MarksInCompressedFileCompact(marks_array, columns.size());
|
||||
}
|
||||
|
||||
const MarkInCompressedFile & MergeTreeReaderCompact::getMark(size_t row, size_t col)
|
||||
{
|
||||
if (!marks.initialized())
|
||||
loadMarks();
|
||||
return marks.getMark(row, col);
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::seekToMark(size_t row, size_t col)
|
||||
{
|
||||
MarkInCompressedFile mark = getMark(row, col);
|
||||
|
||||
try
|
||||
{
|
||||
if (cached_buffer)
|
||||
cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block);
|
||||
if (non_cached_buffer)
|
||||
non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
/// Better diagnostics.
|
||||
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
|
||||
e.addMessage("(while seeking to mark (" + toString(row) + ", " + toString(col) + ")");
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeReaderCompact::seekToStart()
|
||||
{
|
||||
if (cached_buffer)
|
||||
cached_buffer->seek(0, 0);
|
||||
if (non_cached_buffer)
|
||||
non_cached_buffer->seek(0, 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -33,6 +33,8 @@ public:
|
||||
return sizeof(MarkInCompressedFile) * columns_num;
|
||||
}
|
||||
|
||||
bool initialized() { return data != nullptr; }
|
||||
|
||||
private:
|
||||
MarksPtr data;
|
||||
size_t columns_num;
|
||||
@ -58,10 +60,18 @@ public:
|
||||
|
||||
private:
|
||||
ReadBuffer * data_buffer;
|
||||
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
|
||||
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
|
||||
|
||||
MarksInCompressedFileCompact marks;
|
||||
|
||||
void loadMarks();
|
||||
void seekToStart();
|
||||
void seekToMark(size_t row, size_t col);
|
||||
const MarkInCompressedFile & getMark(size_t row, size_t col);
|
||||
|
||||
void readData(const String & name, const IDataType & type, IColumn & column,
|
||||
size_t from_mark, size_t column_position, size_t rows_to_read);
|
||||
|
||||
static auto constexpr NAME_OF_FILE_WITH_DATA = "data";
|
||||
|
||||
|
@ -134,6 +134,8 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si
|
||||
throw;
|
||||
}
|
||||
|
||||
std::cerr << "(MergeTreeReaderWide::readRows) read_rows: " << read_rows << "\n";
|
||||
|
||||
return read_rows;
|
||||
}
|
||||
|
||||
@ -173,6 +175,7 @@ void MergeTreeReaderWide::readData(
|
||||
size_t from_mark, bool continue_reading, size_t max_rows_to_read,
|
||||
bool with_offsets)
|
||||
{
|
||||
std::cerr << "(MergeTreeReaderWide::readData) max_rows_to_read: " << max_rows_to_read << "\n";
|
||||
auto get_stream_getter = [&](bool stream_for_prefix) -> IDataType::InputStreamGetter
|
||||
{
|
||||
return [&, stream_for_prefix](const IDataType::SubstreamPath & substream_path) -> ReadBuffer *
|
||||
|
@ -35,11 +35,9 @@ MergedBlockOutputStream::MergedBlockOutputStream(
|
||||
, columns_list(columns_list_)
|
||||
{
|
||||
init();
|
||||
const auto & columns = storage.getColumns();
|
||||
for (const auto & it : columns_list)
|
||||
{
|
||||
const auto columns = storage.getColumns();
|
||||
addStreams(part_path, it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec_), 0, false);
|
||||
}
|
||||
}
|
||||
|
||||
MergedBlockOutputStream::MergedBlockOutputStream(
|
||||
@ -71,11 +69,9 @@ MergedBlockOutputStream::MergedBlockOutputStream(
|
||||
}
|
||||
}
|
||||
|
||||
const auto & columns = storage.getColumns();
|
||||
for (const auto & it : columns_list)
|
||||
{
|
||||
const auto columns = storage.getColumns();
|
||||
addStreams(part_path, it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec_), total_size, false);
|
||||
}
|
||||
}
|
||||
|
||||
std::string MergedBlockOutputStream::getPartPath() const
|
||||
@ -234,6 +230,7 @@ void MergedBlockOutputStream::init()
|
||||
|
||||
void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Permutation * permutation)
|
||||
{
|
||||
std::cerr << "(MergedBlockOutputStream::writeImpl) block.rows(): " << block.rows() << "\n";
|
||||
block.checkNumberOfRows();
|
||||
size_t rows = block.rows();
|
||||
if (!rows)
|
||||
@ -248,44 +245,41 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
/// The set of written offset columns so that you do not write shared offsets of nested structures columns several times
|
||||
WrittenOffsetColumns offset_columns;
|
||||
|
||||
Block primary_key_block;
|
||||
Block skip_indexes_block;
|
||||
|
||||
auto primary_key_column_names = storage.primary_key_columns;
|
||||
|
||||
std::set<String> skip_indexes_column_names_set;
|
||||
for (const auto & index : storage.skip_indices)
|
||||
std::copy(index->columns.cbegin(), index->columns.cend(),
|
||||
std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end()));
|
||||
Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end());
|
||||
|
||||
/// Here we will add the columns related to the Primary Key, then write the index.
|
||||
std::vector<ColumnWithTypeAndName> primary_key_columns(primary_key_column_names.size());
|
||||
std::map<String, size_t> primary_key_column_name_to_position;
|
||||
|
||||
for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = primary_key_column_names[i];
|
||||
|
||||
if (!primary_key_column_name_to_position.emplace(name, i).second)
|
||||
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
primary_key_columns[i] = block.getByName(name);
|
||||
primary_key_block.insert(i, block.getByName(name));
|
||||
|
||||
/// Reorder primary key columns in advance and add them to `primary_key_columns`.
|
||||
if (permutation)
|
||||
primary_key_columns[i].column = primary_key_columns[i].column->permute(*permutation, 0);
|
||||
{
|
||||
auto & column = primary_key_block.getByPosition(i);
|
||||
column.column = column.column->permute(*permutation, 0);
|
||||
}
|
||||
}
|
||||
|
||||
/// The same for skip indexes columns
|
||||
std::vector<ColumnWithTypeAndName> skip_indexes_columns(skip_indexes_column_names.size());
|
||||
std::map<String, size_t> skip_indexes_column_name_to_position;
|
||||
|
||||
for (size_t i = 0, size = skip_indexes_column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = skip_indexes_column_names[i];
|
||||
skip_indexes_column_name_to_position.emplace(name, i);
|
||||
skip_indexes_columns[i] = block.getByName(name);
|
||||
skip_indexes_block.insert(i, block.getByName(name));
|
||||
|
||||
/// Reorder index columns in advance.
|
||||
if (permutation)
|
||||
skip_indexes_columns[i].column = skip_indexes_columns[i].column->permute(*permutation, 0);
|
||||
{
|
||||
auto & column = skip_indexes_block.getByPosition(i);
|
||||
column.column = column.column->permute(*permutation, 0);
|
||||
}
|
||||
}
|
||||
|
||||
if (index_columns.empty())
|
||||
@ -294,8 +288,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
last_index_row.resize(primary_key_column_names.size());
|
||||
for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i)
|
||||
{
|
||||
index_columns[i] = primary_key_columns[i].column->cloneEmpty();
|
||||
last_index_row[i] = primary_key_columns[i].cloneEmpty();
|
||||
last_index_row[i] = primary_key_block.getByPosition(i).cloneEmpty();
|
||||
index_columns[i] = last_index_row[i].column->cloneEmpty();
|
||||
}
|
||||
}
|
||||
|
||||
@ -313,6 +307,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
}
|
||||
}
|
||||
|
||||
writer->write(block, primary_key_block, skip_indexes_block, current_mark, index_offset);
|
||||
|
||||
size_t new_index_offset = 0;
|
||||
/// Now write the data.
|
||||
auto it = columns_list.begin();
|
||||
@ -322,9 +318,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
|
||||
if (permutation)
|
||||
{
|
||||
auto primary_column_it = primary_key_column_name_to_position.find(it->name);
|
||||
auto skip_index_column_it = skip_indexes_column_name_to_position.find(it->name);
|
||||
if (primary_key_column_name_to_position.end() != primary_column_it)
|
||||
if (primary_key_block.has(it->name))
|
||||
{
|
||||
const auto & primary_column = *primary_key_columns[primary_column_it->second].column;
|
||||
std::tie(std::ignore, new_index_offset) = writeColumn(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i], current_mark);
|
||||
@ -347,6 +341,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
}
|
||||
}
|
||||
|
||||
std::cerr << "(MergedBlockOutputStream::writeImpl) new_index_offset: " << new_index_offset << "\n";
|
||||
|
||||
rows_count += rows;
|
||||
|
||||
/// Should be written before index offset update, because we calculate,
|
||||
|
Loading…
Reference in New Issue
Block a user