mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Keep indices for StorageStripeLog in memory.
This commit is contained in:
parent
42596b16bc
commit
0e8c9b089f
@ -50,11 +50,6 @@ void FileChecker::setEmpty(const String & full_file_path)
|
|||||||
map[fileName(full_file_path)] = 0;
|
map[fileName(full_file_path)] = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
const FileChecker::Map & FileChecker::getFileSizes() const
|
|
||||||
{
|
|
||||||
return map;
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t FileChecker::getFileSize(const String & full_file_path) const
|
size_t FileChecker::getFileSize(const String & full_file_path) const
|
||||||
{
|
{
|
||||||
auto it = map.find(fileName(full_file_path));
|
auto it = map.find(fileName(full_file_path));
|
||||||
|
@ -28,11 +28,6 @@ public:
|
|||||||
/// The purpose of this function is to rollback a group of unfinished writes.
|
/// The purpose of this function is to rollback a group of unfinished writes.
|
||||||
void repair();
|
void repair();
|
||||||
|
|
||||||
/// File name -> size.
|
|
||||||
using Map = std::map<String, UInt64>;
|
|
||||||
|
|
||||||
const Map & getFileSizes() const;
|
|
||||||
|
|
||||||
/// Returns stored file size.
|
/// Returns stored file size.
|
||||||
size_t getFileSize(const String & full_file_path) const;
|
size_t getFileSize(const String & full_file_path) const;
|
||||||
|
|
||||||
@ -43,7 +38,7 @@ private:
|
|||||||
const Poco::Logger * log = &Poco::Logger::get("FileChecker");
|
const Poco::Logger * log = &Poco::Logger::get("FileChecker");
|
||||||
|
|
||||||
String files_info_path;
|
String files_info_path;
|
||||||
Map map;
|
std::map<String, size_t> map;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
91
src/Formats/IndexForNativeFormat.cpp
Normal file
91
src/Formats/IndexForNativeFormat.cpp
Normal file
@ -0,0 +1,91 @@
|
|||||||
|
#include <Formats/IndexForNativeFormat.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int INCORRECT_INDEX;
|
||||||
|
}
|
||||||
|
|
||||||
|
void IndexOfBlockForNativeFormat::read(ReadBuffer & istr)
|
||||||
|
{
|
||||||
|
readVarUInt(num_columns, istr);
|
||||||
|
readVarUInt(num_rows, istr);
|
||||||
|
columns.clear();
|
||||||
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
|
{
|
||||||
|
auto & column = columns.emplace_back();
|
||||||
|
readBinary(column.name, istr);
|
||||||
|
readBinary(column.type, istr);
|
||||||
|
readBinary(column.location.offset_in_compressed_file, istr);
|
||||||
|
readBinary(column.location.offset_in_decompressed_block, istr);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void IndexOfBlockForNativeFormat::write(WriteBuffer & ostr) const
|
||||||
|
{
|
||||||
|
writeVarUInt(num_columns, ostr);
|
||||||
|
writeVarUInt(num_rows, ostr);
|
||||||
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
|
{
|
||||||
|
const auto & column = columns[i];
|
||||||
|
writeBinary(column.name, ostr);
|
||||||
|
writeBinary(column.type, ostr);
|
||||||
|
writeBinary(column.location.offset_in_compressed_file, ostr);
|
||||||
|
writeBinary(column.location.offset_in_decompressed_block, ostr);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexOfBlockForNativeFormat IndexOfBlockForNativeFormat::extractIndexForColumns(const NameSet & required_columns) const
|
||||||
|
{
|
||||||
|
if (num_columns < required_columns.size())
|
||||||
|
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
|
||||||
|
|
||||||
|
IndexOfBlockForNativeFormat res;
|
||||||
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
|
{
|
||||||
|
const auto & column = columns[i];
|
||||||
|
if (required_columns.contains(column.name))
|
||||||
|
res.columns.push_back(column);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (res.columns.size() < required_columns.size())
|
||||||
|
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
|
||||||
|
if (res.columns.size() > required_columns.size())
|
||||||
|
throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX);
|
||||||
|
|
||||||
|
res.num_columns = res.columns.size();
|
||||||
|
res.num_rows = num_rows;
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IndexForNativeFormat::read(ReadBuffer & istr)
|
||||||
|
{
|
||||||
|
blocks.clear();
|
||||||
|
while (!istr.eof())
|
||||||
|
{
|
||||||
|
auto & block = blocks.emplace_back();
|
||||||
|
block.read(istr);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void IndexForNativeFormat::write(WriteBuffer & ostr) const
|
||||||
|
{
|
||||||
|
for (const auto & block : blocks)
|
||||||
|
block.write(ostr);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexForNativeFormat IndexForNativeFormat::extractIndexForColumns(const NameSet & required_columns) const
|
||||||
|
{
|
||||||
|
IndexForNativeFormat res;
|
||||||
|
res.blocks.reserve(blocks.size());
|
||||||
|
for (const auto & block : blocks)
|
||||||
|
res.blocks.emplace_back(block.extractIndexForColumns(required_columns));
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
60
src/Formats/IndexForNativeFormat.h
Normal file
60
src/Formats/IndexForNativeFormat.h
Normal file
@ -0,0 +1,60 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Core/Names.h>
|
||||||
|
#include <Formats/MarkInCompressedFile.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/** The Native format can contain a separately located index,
|
||||||
|
* which allows you to understand where what column is located,
|
||||||
|
* and skip unnecessary columns.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/** The position of one piece of a single column. */
|
||||||
|
struct IndexOfOneColumnForNativeFormat
|
||||||
|
{
|
||||||
|
String name;
|
||||||
|
String type;
|
||||||
|
MarkInCompressedFile location;
|
||||||
|
};
|
||||||
|
|
||||||
|
/** The index for the data block. */
|
||||||
|
struct IndexOfBlockForNativeFormat
|
||||||
|
{
|
||||||
|
using Columns = std::vector<IndexOfOneColumnForNativeFormat>;
|
||||||
|
|
||||||
|
size_t num_columns;
|
||||||
|
size_t num_rows;
|
||||||
|
Columns columns;
|
||||||
|
|
||||||
|
/// Reads the index for the data block.
|
||||||
|
void read(ReadBuffer & istr);
|
||||||
|
|
||||||
|
/// Writes the index for the data block.
|
||||||
|
void write(WriteBuffer & ostr) const;
|
||||||
|
|
||||||
|
/// Returns the index only for the required columns.
|
||||||
|
IndexOfBlockForNativeFormat extractIndexForColumns(const NameSet & required_columns) const;
|
||||||
|
};
|
||||||
|
|
||||||
|
/** The whole index. */
|
||||||
|
struct IndexForNativeFormat
|
||||||
|
{
|
||||||
|
using Blocks = std::vector<IndexOfBlockForNativeFormat>;
|
||||||
|
Blocks blocks;
|
||||||
|
|
||||||
|
bool empty() const { return blocks.empty(); }
|
||||||
|
void clear() { blocks.clear(); }
|
||||||
|
|
||||||
|
/// Reads the index.
|
||||||
|
void read(ReadBuffer & istr);
|
||||||
|
|
||||||
|
/// Writes the index.
|
||||||
|
void write(WriteBuffer & ostr) const;
|
||||||
|
|
||||||
|
/// Returns the index only for the required columns.
|
||||||
|
IndexForNativeFormat extractIndexForColumns(const NameSet & required_columns) const;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -221,39 +221,4 @@ void NativeReader::updateAvgValueSizeHints(const Block & block)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void IndexForNativeFormat::read(ReadBuffer & istr, const NameSet & required_columns)
|
|
||||||
{
|
|
||||||
while (!istr.eof())
|
|
||||||
{
|
|
||||||
blocks.emplace_back();
|
|
||||||
IndexOfBlockForNativeFormat & block = blocks.back();
|
|
||||||
|
|
||||||
readVarUInt(block.num_columns, istr);
|
|
||||||
readVarUInt(block.num_rows, istr);
|
|
||||||
|
|
||||||
if (block.num_columns < required_columns.size())
|
|
||||||
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < block.num_columns; ++i)
|
|
||||||
{
|
|
||||||
IndexOfOneColumnForNativeFormat column_index;
|
|
||||||
|
|
||||||
readBinary(column_index.name, istr);
|
|
||||||
readBinary(column_index.type, istr);
|
|
||||||
readBinary(column_index.location.offset_in_compressed_file, istr);
|
|
||||||
readBinary(column_index.location.offset_in_decompressed_block, istr);
|
|
||||||
|
|
||||||
if (required_columns.count(column_index.name))
|
|
||||||
block.columns.push_back(std::move(column_index));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (block.columns.size() < required_columns.size())
|
|
||||||
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
|
|
||||||
if (block.columns.size() > required_columns.size())
|
|
||||||
throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX);
|
|
||||||
|
|
||||||
block.num_columns = block.columns.size();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Formats/IndexForNativeFormat.h>
|
||||||
#include <Formats/MarkInCompressedFile.h>
|
#include <Formats/MarkInCompressedFile.h>
|
||||||
#include <Common/PODArray.h>
|
#include <Common/PODArray.h>
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
@ -9,48 +10,6 @@ namespace DB
|
|||||||
|
|
||||||
class CompressedReadBufferFromFile;
|
class CompressedReadBufferFromFile;
|
||||||
|
|
||||||
|
|
||||||
/** The Native format can contain a separately located index,
|
|
||||||
* which allows you to understand where what column is located,
|
|
||||||
* and skip unnecessary columns.
|
|
||||||
*/
|
|
||||||
|
|
||||||
/** The position of one piece of a single column. */
|
|
||||||
struct IndexOfOneColumnForNativeFormat
|
|
||||||
{
|
|
||||||
String name;
|
|
||||||
String type;
|
|
||||||
MarkInCompressedFile location;
|
|
||||||
};
|
|
||||||
|
|
||||||
/** The index for the data block. */
|
|
||||||
struct IndexOfBlockForNativeFormat
|
|
||||||
{
|
|
||||||
using Columns = std::vector<IndexOfOneColumnForNativeFormat>;
|
|
||||||
|
|
||||||
size_t num_columns;
|
|
||||||
size_t num_rows;
|
|
||||||
Columns columns;
|
|
||||||
};
|
|
||||||
|
|
||||||
/** The whole index. */
|
|
||||||
struct IndexForNativeFormat
|
|
||||||
{
|
|
||||||
using Blocks = std::vector<IndexOfBlockForNativeFormat>;
|
|
||||||
Blocks blocks;
|
|
||||||
|
|
||||||
IndexForNativeFormat() {}
|
|
||||||
|
|
||||||
IndexForNativeFormat(ReadBuffer & istr, const NameSet & required_columns)
|
|
||||||
{
|
|
||||||
read(istr, required_columns);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Read the index, only for the required columns.
|
|
||||||
void read(ReadBuffer & istr, const NameSet & required_columns);
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
/** Deserializes the stream of blocks from the native binary format (with names and column types).
|
/** Deserializes the stream of blocks from the native binary format (with names and column types).
|
||||||
* Designed for communication between servers.
|
* Designed for communication between servers.
|
||||||
*
|
*
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <IO/VarInt.h>
|
#include <IO/VarInt.h>
|
||||||
#include <Compression/CompressedWriteBuffer.h>
|
#include <Compression/CompressedWriteBuffer.h>
|
||||||
|
|
||||||
|
#include <Formats/IndexForNativeFormat.h>
|
||||||
#include <Formats/MarkInCompressedFile.h>
|
#include <Formats/MarkInCompressedFile.h>
|
||||||
#include <Formats/NativeWriter.h>
|
#include <Formats/NativeWriter.h>
|
||||||
|
|
||||||
@ -22,11 +23,11 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
NativeWriter::NativeWriter(
|
NativeWriter::NativeWriter(
|
||||||
WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_,
|
WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_,
|
||||||
WriteBuffer * index_ostr_, size_t initial_size_of_file_)
|
IndexForNativeFormat * index_, size_t initial_size_of_file_)
|
||||||
: ostr(ostr_), client_revision(client_revision_), header(header_),
|
: ostr(ostr_), client_revision(client_revision_), header(header_),
|
||||||
index_ostr(index_ostr_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_)
|
index(index_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_)
|
||||||
{
|
{
|
||||||
if (index_ostr)
|
if (index)
|
||||||
{
|
{
|
||||||
ostr_concrete = typeid_cast<CompressedWriteBuffer *>(&ostr);
|
ostr_concrete = typeid_cast<CompressedWriteBuffer *>(&ostr);
|
||||||
if (!ostr_concrete)
|
if (!ostr_concrete)
|
||||||
@ -80,18 +81,20 @@ void NativeWriter::write(const Block & block)
|
|||||||
/** The index has the same structure as the data stream.
|
/** The index has the same structure as the data stream.
|
||||||
* But instead of column values, it contains a mark that points to the location in the data file where this part of the column is located.
|
* But instead of column values, it contains a mark that points to the location in the data file where this part of the column is located.
|
||||||
*/
|
*/
|
||||||
if (index_ostr)
|
IndexOfBlockForNativeFormat index_block;
|
||||||
|
if (index)
|
||||||
{
|
{
|
||||||
writeVarUInt(columns, *index_ostr);
|
index_block.num_columns = columns;
|
||||||
writeVarUInt(rows, *index_ostr);
|
index_block.num_rows = rows;
|
||||||
|
index_block.columns.resize(columns);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (size_t i = 0; i < columns; ++i)
|
for (size_t i = 0; i < columns; ++i)
|
||||||
{
|
{
|
||||||
/// For the index.
|
/// For the index.
|
||||||
MarkInCompressedFile mark;
|
MarkInCompressedFile mark{0, 0};
|
||||||
|
|
||||||
if (index_ostr)
|
if (index)
|
||||||
{
|
{
|
||||||
ostr_concrete->next(); /// Finish compressed block.
|
ostr_concrete->next(); /// Finish compressed block.
|
||||||
mark.offset_in_compressed_file = initial_size_of_file + ostr_concrete->getCompressedBytes();
|
mark.offset_in_compressed_file = initial_size_of_file + ostr_concrete->getCompressedBytes();
|
||||||
@ -125,15 +128,17 @@ void NativeWriter::write(const Block & block)
|
|||||||
if (rows) /// Zero items of data is always represented as zero number of bytes.
|
if (rows) /// Zero items of data is always represented as zero number of bytes.
|
||||||
writeData(*column.type, column.column, ostr, 0, 0);
|
writeData(*column.type, column.column, ostr, 0, 0);
|
||||||
|
|
||||||
if (index_ostr)
|
if (index)
|
||||||
{
|
{
|
||||||
writeStringBinary(column.name, *index_ostr);
|
index_block.columns[i].name = column.name;
|
||||||
writeStringBinary(column.type->getName(), *index_ostr);
|
index_block.columns[i].type = column.type->getName();
|
||||||
|
index_block.columns[i].location.offset_in_compressed_file = mark.offset_in_compressed_file;
|
||||||
writeBinary(mark.offset_in_compressed_file, *index_ostr);
|
index_block.columns[i].location.offset_in_decompressed_block = mark.offset_in_decompressed_block;
|
||||||
writeBinary(mark.offset_in_decompressed_block, *index_ostr);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (index)
|
||||||
|
index->blocks.emplace_back(std::move(index_block));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,7 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
class CompressedWriteBuffer;
|
class CompressedWriteBuffer;
|
||||||
|
struct IndexForNativeFormat;
|
||||||
|
|
||||||
/** Serializes the stream of blocks in their native binary format (with names and column types).
|
/** Serializes the stream of blocks in their native binary format (with names and column types).
|
||||||
* Designed for communication between servers.
|
* Designed for communication between servers.
|
||||||
@ -24,7 +24,7 @@ public:
|
|||||||
*/
|
*/
|
||||||
NativeWriter(
|
NativeWriter(
|
||||||
WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false,
|
WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false,
|
||||||
WriteBuffer * index_ostr_ = nullptr, size_t initial_size_of_file_ = 0);
|
IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0);
|
||||||
|
|
||||||
Block getHeader() const { return header; }
|
Block getHeader() const { return header; }
|
||||||
void write(const Block & block);
|
void write(const Block & block);
|
||||||
@ -36,7 +36,7 @@ private:
|
|||||||
WriteBuffer & ostr;
|
WriteBuffer & ostr;
|
||||||
UInt64 client_revision;
|
UInt64 client_revision;
|
||||||
Block header;
|
Block header;
|
||||||
WriteBuffer * index_ostr;
|
IndexForNativeFormat * index = nullptr;
|
||||||
size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index.
|
size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index.
|
||||||
/// If you need to write index, then `ostr` must be a CompressedWriteBuffer.
|
/// If you need to write index, then `ostr` must be a CompressedWriteBuffer.
|
||||||
CompressedWriteBuffer * ostr_concrete = nullptr;
|
CompressedWriteBuffer * ostr_concrete = nullptr;
|
||||||
|
@ -47,11 +47,13 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/// NOTE: The lock `StorageStripeLog::rwlock` is NOT kept locked while reading,
|
||||||
|
/// because we read ranges of data that do not change.
|
||||||
class StripeLogSource final : public SourceWithProgress
|
class StripeLogSource final : public SourceWithProgress
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static Block getHeader(
|
static Block getHeader(
|
||||||
StorageStripeLog & storage,
|
const StorageStripeLog & storage,
|
||||||
const StorageMetadataPtr & metadata_snapshot,
|
const StorageMetadataPtr & metadata_snapshot,
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
IndexForNativeFormat::Blocks::const_iterator index_begin,
|
IndexForNativeFormat::Blocks::const_iterator index_begin,
|
||||||
@ -74,19 +76,18 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
StripeLogSource(
|
StripeLogSource(
|
||||||
StorageStripeLog & storage_,
|
const StorageStripeLog & storage_,
|
||||||
const StorageMetadataPtr & metadata_snapshot_,
|
const StorageMetadataPtr & metadata_snapshot_,
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
ReadSettings read_settings_,
|
ReadSettings read_settings_,
|
||||||
std::shared_ptr<const IndexForNativeFormat> & index_,
|
std::shared_ptr<const IndexForNativeFormat> indices_,
|
||||||
IndexForNativeFormat::Blocks::const_iterator index_begin_,
|
IndexForNativeFormat::Blocks::const_iterator index_begin_,
|
||||||
IndexForNativeFormat::Blocks::const_iterator index_end_)
|
IndexForNativeFormat::Blocks::const_iterator index_end_)
|
||||||
: SourceWithProgress(
|
: SourceWithProgress(getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_))
|
||||||
getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_))
|
|
||||||
, storage(storage_)
|
, storage(storage_)
|
||||||
, metadata_snapshot(metadata_snapshot_)
|
, metadata_snapshot(metadata_snapshot_)
|
||||||
, read_settings(std::move(read_settings_))
|
, read_settings(std::move(read_settings_))
|
||||||
, index(index_)
|
, indices(indices_)
|
||||||
, index_begin(index_begin_)
|
, index_begin(index_begin_)
|
||||||
, index_end(index_end_)
|
, index_end(index_end_)
|
||||||
{
|
{
|
||||||
@ -109,7 +110,7 @@ protected:
|
|||||||
{
|
{
|
||||||
block_in.reset();
|
block_in.reset();
|
||||||
data_in.reset();
|
data_in.reset();
|
||||||
index.reset();
|
indices.reset();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -117,13 +118,14 @@ protected:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
StorageStripeLog & storage;
|
const StorageStripeLog & storage;
|
||||||
StorageMetadataPtr metadata_snapshot;
|
StorageMetadataPtr metadata_snapshot;
|
||||||
ReadSettings read_settings;
|
ReadSettings read_settings;
|
||||||
|
|
||||||
std::shared_ptr<const IndexForNativeFormat> index;
|
std::shared_ptr<const IndexForNativeFormat> indices;
|
||||||
IndexForNativeFormat::Blocks::const_iterator index_begin;
|
IndexForNativeFormat::Blocks::const_iterator index_begin;
|
||||||
IndexForNativeFormat::Blocks::const_iterator index_end;
|
IndexForNativeFormat::Blocks::const_iterator index_end;
|
||||||
|
|
||||||
Block header;
|
Block header;
|
||||||
|
|
||||||
/** optional - to create objects only on first reading
|
/** optional - to create objects only on first reading
|
||||||
@ -141,40 +143,45 @@ private:
|
|||||||
started = true;
|
started = true;
|
||||||
|
|
||||||
String data_file_path = storage.table_path + "data.bin";
|
String data_file_path = storage.table_path + "data.bin";
|
||||||
data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(storage.disk->getFileSize(data_file_path))));
|
|
||||||
|
/// We cannot just use `storage.file_checker` to get the size of the file here,
|
||||||
|
/// because `storage.rwlock` is not locked at this point.
|
||||||
|
size_t data_file_size = storage.disk->getFileSize(data_file_path);
|
||||||
|
|
||||||
|
data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(data_file_size)));
|
||||||
block_in.emplace(*data_in, 0, index_begin, index_end);
|
block_in.emplace(*data_in, 0, index_begin, index_end);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/// NOTE: The lock `StorageStripeLog::rwlock` is kept locked in exclusive mode while writing.
|
||||||
class StripeLogSink final : public SinkToStorage
|
class StripeLogSink final : public SinkToStorage
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
using WriteLock = std::unique_lock<std::shared_timed_mutex>;
|
||||||
|
|
||||||
explicit StripeLogSink(
|
explicit StripeLogSink(
|
||||||
StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock<std::shared_timed_mutex> && lock_)
|
StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, WriteLock && lock_)
|
||||||
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
||||||
, storage(storage_)
|
, storage(storage_)
|
||||||
, metadata_snapshot(metadata_snapshot_)
|
, metadata_snapshot(metadata_snapshot_)
|
||||||
, lock(std::move(lock_))
|
, lock(std::move(lock_))
|
||||||
, data_out_file(storage.table_path + "data.bin")
|
, data_out_compressed(storage.disk->writeFile(storage.data_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append))
|
||||||
, data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append))
|
|
||||||
, data_out(std::make_unique<CompressedWriteBuffer>(
|
, data_out(std::make_unique<CompressedWriteBuffer>(
|
||||||
*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size))
|
*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size))
|
||||||
, index_out_file(storage.table_path + "index.mrk")
|
|
||||||
, index_out_compressed(storage.disk->writeFile(index_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append))
|
|
||||||
, index_out(std::make_unique<CompressedWriteBuffer>(*index_out_compressed))
|
|
||||||
, block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file))
|
|
||||||
{
|
{
|
||||||
if (!lock)
|
if (!lock)
|
||||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||||
|
|
||||||
if (storage.file_checker.empty())
|
/// Ensure that indices are loaded because we're going to update them.
|
||||||
{
|
storage.loadIndices(lock);
|
||||||
storage.file_checker.setEmpty(storage.table_path + "data.bin");
|
|
||||||
storage.file_checker.setEmpty(storage.table_path + "index.mrk");
|
/// If there were no files, save zero file sizes to be able to rollback in case of error.
|
||||||
storage.file_checker.save();
|
storage.saveFileSizes(lock);
|
||||||
}
|
|
||||||
|
size_t initial_data_size = storage.file_checker.getFileSize(storage.data_file_path);
|
||||||
|
block_out = std::make_unique<NativeWriter>(*data_out, 0, metadata_snapshot->getSampleBlock(), false, &storage.indices, initial_data_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
String getName() const override { return "StripeLogSink"; }
|
String getName() const override { return "StripeLogSink"; }
|
||||||
@ -186,12 +193,16 @@ public:
|
|||||||
if (!done)
|
if (!done)
|
||||||
{
|
{
|
||||||
/// Rollback partial writes.
|
/// Rollback partial writes.
|
||||||
|
|
||||||
|
/// No more writing.
|
||||||
data_out.reset();
|
data_out.reset();
|
||||||
data_out_compressed.reset();
|
data_out_compressed.reset();
|
||||||
index_out.reset();
|
|
||||||
index_out_compressed.reset();
|
|
||||||
|
|
||||||
|
/// Truncate files to the older sizes.
|
||||||
storage.file_checker.repair();
|
storage.file_checker.repair();
|
||||||
|
|
||||||
|
/// Remove excessive indices.
|
||||||
|
storage.removeUnsavedIndices(lock);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
@ -202,7 +213,7 @@ public:
|
|||||||
|
|
||||||
void consume(Chunk chunk) override
|
void consume(Chunk chunk) override
|
||||||
{
|
{
|
||||||
block_out.write(getHeader().cloneWithColumns(chunk.detachColumns()));
|
block_out->write(getHeader().cloneWithColumns(chunk.detachColumns()));
|
||||||
}
|
}
|
||||||
|
|
||||||
void onFinish() override
|
void onFinish() override
|
||||||
@ -213,13 +224,12 @@ public:
|
|||||||
data_out->next();
|
data_out->next();
|
||||||
data_out_compressed->next();
|
data_out_compressed->next();
|
||||||
data_out_compressed->finalize();
|
data_out_compressed->finalize();
|
||||||
index_out->next();
|
|
||||||
index_out_compressed->next();
|
|
||||||
index_out_compressed->finalize();
|
|
||||||
|
|
||||||
storage.file_checker.update(data_out_file);
|
/// Save the new indices.
|
||||||
storage.file_checker.update(index_out_file);
|
storage.saveIndices(lock);
|
||||||
storage.file_checker.save();
|
|
||||||
|
/// Save the new file sizes.
|
||||||
|
storage.saveFileSizes(lock);
|
||||||
|
|
||||||
done = true;
|
done = true;
|
||||||
|
|
||||||
@ -232,15 +242,11 @@ public:
|
|||||||
private:
|
private:
|
||||||
StorageStripeLog & storage;
|
StorageStripeLog & storage;
|
||||||
StorageMetadataPtr metadata_snapshot;
|
StorageMetadataPtr metadata_snapshot;
|
||||||
std::unique_lock<std::shared_timed_mutex> lock;
|
WriteLock lock;
|
||||||
|
|
||||||
String data_out_file;
|
|
||||||
std::unique_ptr<WriteBuffer> data_out_compressed;
|
std::unique_ptr<WriteBuffer> data_out_compressed;
|
||||||
std::unique_ptr<CompressedWriteBuffer> data_out;
|
std::unique_ptr<CompressedWriteBuffer> data_out;
|
||||||
String index_out_file;
|
std::unique_ptr<NativeWriter> block_out;
|
||||||
std::unique_ptr<WriteBuffer> index_out_compressed;
|
|
||||||
std::unique_ptr<CompressedWriteBuffer> index_out;
|
|
||||||
NativeWriter block_out;
|
|
||||||
|
|
||||||
bool done = false;
|
bool done = false;
|
||||||
};
|
};
|
||||||
@ -258,8 +264,10 @@ StorageStripeLog::StorageStripeLog(
|
|||||||
: IStorage(table_id_)
|
: IStorage(table_id_)
|
||||||
, disk(std::move(disk_))
|
, disk(std::move(disk_))
|
||||||
, table_path(relative_path_)
|
, table_path(relative_path_)
|
||||||
, max_compress_block_size(max_compress_block_size_)
|
, data_file_path(table_path + "data.bin")
|
||||||
|
, index_file_path(table_path + "index.mrk")
|
||||||
, file_checker(disk, table_path + "sizes.json")
|
, file_checker(disk, table_path + "sizes.json")
|
||||||
|
, max_compress_block_size(max_compress_block_size_)
|
||||||
, log(&Poco::Logger::get("StorageStripeLog"))
|
, log(&Poco::Logger::get("StorageStripeLog"))
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
@ -271,6 +279,13 @@ StorageStripeLog::StorageStripeLog(
|
|||||||
if (relative_path_.empty())
|
if (relative_path_.empty())
|
||||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||||
|
|
||||||
|
/// Ensure the file checker is initialized.
|
||||||
|
if (file_checker.empty())
|
||||||
|
{
|
||||||
|
file_checker.setEmpty(data_file_path);
|
||||||
|
file_checker.setEmpty(index_file_path);
|
||||||
|
}
|
||||||
|
|
||||||
if (!attach)
|
if (!attach)
|
||||||
{
|
{
|
||||||
/// create directories if they do not exist
|
/// create directories if they do not exist
|
||||||
@ -290,6 +305,9 @@ StorageStripeLog::StorageStripeLog(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
StorageStripeLog::~StorageStripeLog() = default;
|
||||||
|
|
||||||
|
|
||||||
void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
assert(table_path != new_path_to_table_data);
|
assert(table_path != new_path_to_table_data);
|
||||||
@ -297,6 +315,8 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora
|
|||||||
disk->moveDirectory(table_path, new_path_to_table_data);
|
disk->moveDirectory(table_path, new_path_to_table_data);
|
||||||
|
|
||||||
table_path = new_path_to_table_data;
|
table_path = new_path_to_table_data;
|
||||||
|
data_file_path = table_path + "data.bin";
|
||||||
|
index_file_path = table_path + "index.mrk";
|
||||||
file_checker.setPath(table_path + "sizes.json");
|
file_checker.setPath(table_path + "sizes.json");
|
||||||
}
|
}
|
||||||
renameInMemory(new_table_id);
|
renameInMemory(new_table_id);
|
||||||
@ -322,41 +342,38 @@ Pipe StorageStripeLog::read(
|
|||||||
const size_t /*max_block_size*/,
|
const size_t /*max_block_size*/,
|
||||||
unsigned num_streams)
|
unsigned num_streams)
|
||||||
{
|
{
|
||||||
std::shared_lock lock(rwlock, getLockTimeout(context));
|
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
||||||
|
|
||||||
|
auto lock_timeout = getLockTimeout(context);
|
||||||
|
loadIndices(lock_timeout);
|
||||||
|
|
||||||
|
ReadLock lock{rwlock, lock_timeout};
|
||||||
if (!lock)
|
if (!lock)
|
||||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||||
|
|
||||||
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
if (!file_checker.getFileSize(data_file_path))
|
||||||
|
|
||||||
NameSet column_names_set(column_names.begin(), column_names.end());
|
|
||||||
|
|
||||||
Pipes pipes;
|
|
||||||
|
|
||||||
String index_file = table_path + "index.mrk";
|
|
||||||
if (file_checker.empty() || !disk->exists(index_file))
|
|
||||||
{
|
|
||||||
return Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
|
return Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
|
||||||
}
|
|
||||||
|
|
||||||
ReadSettings read_settings = context->getReadSettings();
|
auto indices_for_selected_columns
|
||||||
|
= std::make_shared<IndexForNativeFormat>(indices.extractIndexForColumns(NameSet{column_names.begin(), column_names.end()}));
|
||||||
|
|
||||||
CompressedReadBufferFromFile index_in(disk->readFile(index_file, read_settings.adjustBufferSize(4096)));
|
size_t size = indices_for_selected_columns->blocks.size();
|
||||||
std::shared_ptr<const IndexForNativeFormat> index{std::make_shared<IndexForNativeFormat>(index_in, column_names_set)};
|
|
||||||
|
|
||||||
size_t size = index->blocks.size();
|
|
||||||
if (num_streams > size)
|
if (num_streams > size)
|
||||||
num_streams = size;
|
num_streams = size;
|
||||||
|
|
||||||
|
ReadSettings read_settings = context->getReadSettings();
|
||||||
|
Pipes pipes;
|
||||||
|
|
||||||
for (size_t stream = 0; stream < num_streams; ++stream)
|
for (size_t stream = 0; stream < num_streams; ++stream)
|
||||||
{
|
{
|
||||||
IndexForNativeFormat::Blocks::const_iterator begin = index->blocks.begin();
|
IndexForNativeFormat::Blocks::const_iterator begin = indices_for_selected_columns->blocks.begin();
|
||||||
IndexForNativeFormat::Blocks::const_iterator end = index->blocks.begin();
|
IndexForNativeFormat::Blocks::const_iterator end = indices_for_selected_columns->blocks.begin();
|
||||||
|
|
||||||
std::advance(begin, stream * size / num_streams);
|
std::advance(begin, stream * size / num_streams);
|
||||||
std::advance(end, (stream + 1) * size / num_streams);
|
std::advance(end, (stream + 1) * size / num_streams);
|
||||||
|
|
||||||
pipes.emplace_back(std::make_shared<StripeLogSource>(
|
pipes.emplace_back(std::make_shared<StripeLogSource>(
|
||||||
*this, metadata_snapshot, column_names, read_settings, index, begin, end));
|
*this, metadata_snapshot, column_names, read_settings, indices_for_selected_columns, begin, end));
|
||||||
}
|
}
|
||||||
|
|
||||||
/// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change.
|
/// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change.
|
||||||
@ -367,7 +384,7 @@ Pipe StorageStripeLog::read(
|
|||||||
|
|
||||||
SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
||||||
{
|
{
|
||||||
std::unique_lock lock(rwlock, getLockTimeout(context));
|
WriteLock lock{rwlock, getLockTimeout(context)};
|
||||||
if (!lock)
|
if (!lock)
|
||||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||||
|
|
||||||
@ -377,17 +394,91 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage
|
|||||||
|
|
||||||
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr context)
|
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr context)
|
||||||
{
|
{
|
||||||
std::shared_lock lock(rwlock, getLockTimeout(context));
|
ReadLock lock{rwlock, getLockTimeout(context)};
|
||||||
if (!lock)
|
if (!lock)
|
||||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||||
|
|
||||||
return file_checker.check();
|
return file_checker.check();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
|
void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
|
||||||
{
|
{
|
||||||
disk->clearDirectory(table_path);
|
disk->clearDirectory(table_path);
|
||||||
file_checker = FileChecker{disk, table_path + "sizes.json"};
|
|
||||||
|
indices.clear();
|
||||||
|
file_checker.setEmpty(data_file_path);
|
||||||
|
file_checker.setEmpty(index_file_path);
|
||||||
|
|
||||||
|
indices_loaded = true;
|
||||||
|
num_indices_saved = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageStripeLog::loadIndices(std::chrono::seconds lock_timeout)
|
||||||
|
{
|
||||||
|
if (indices_loaded)
|
||||||
|
return;
|
||||||
|
|
||||||
|
/// We load indices with an exclusive lock (i.e. the write lock) because we don't want
|
||||||
|
/// a data race between two threads trying to load indices simultaneously.
|
||||||
|
WriteLock lock{rwlock, lock_timeout};
|
||||||
|
if (!lock)
|
||||||
|
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||||
|
|
||||||
|
loadIndices(lock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageStripeLog::loadIndices(const WriteLock & /* already locked exclusively */)
|
||||||
|
{
|
||||||
|
if (indices_loaded)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (disk->exists(index_file_path))
|
||||||
|
{
|
||||||
|
CompressedReadBufferFromFile index_in(disk->readFile(index_file_path, ReadSettings{}.adjustBufferSize(4096)));
|
||||||
|
indices.read(index_in);
|
||||||
|
}
|
||||||
|
|
||||||
|
indices_loaded = true;
|
||||||
|
num_indices_saved = indices.blocks.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageStripeLog::saveIndices(const WriteLock & /* already locked for writing */)
|
||||||
|
{
|
||||||
|
size_t num_indices = indices.blocks.size();
|
||||||
|
if (num_indices_saved == num_indices)
|
||||||
|
return;
|
||||||
|
|
||||||
|
size_t start = num_indices_saved;
|
||||||
|
auto index_out_compressed = disk->writeFile(index_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append);
|
||||||
|
auto index_out = std::make_unique<CompressedWriteBuffer>(*index_out_compressed);
|
||||||
|
|
||||||
|
for (size_t i = start; i != num_indices; ++i)
|
||||||
|
indices.blocks[i].write(*index_out);
|
||||||
|
|
||||||
|
index_out->next();
|
||||||
|
index_out_compressed->next();
|
||||||
|
index_out_compressed->finalize();
|
||||||
|
|
||||||
|
num_indices_saved = num_indices;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageStripeLog::removeUnsavedIndices(const WriteLock & /* already locked for writing */)
|
||||||
|
{
|
||||||
|
if (indices.blocks.size() > num_indices_saved)
|
||||||
|
indices.blocks.resize(num_indices_saved);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for writing */)
|
||||||
|
{
|
||||||
|
file_checker.update(data_file_path);
|
||||||
|
file_checker.update(index_file_path);
|
||||||
|
file_checker.save();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -7,12 +7,15 @@
|
|||||||
|
|
||||||
#include <Core/Defines.h>
|
#include <Core/Defines.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
|
#include <Formats/IndexForNativeFormat.h>
|
||||||
#include <Common/FileChecker.h>
|
#include <Common/FileChecker.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct IndexForNativeFormat;
|
||||||
|
|
||||||
/** Implements a table engine that is suitable for small chunks of the log.
|
/** Implements a table engine that is suitable for small chunks of the log.
|
||||||
* In doing so, stores all the columns in a single Native file, with a nearby index.
|
* In doing so, stores all the columns in a single Native file, with a nearby index.
|
||||||
*/
|
*/
|
||||||
@ -23,6 +26,8 @@ class StorageStripeLog final : public shared_ptr_helper<StorageStripeLog>, publi
|
|||||||
friend struct shared_ptr_helper<StorageStripeLog>;
|
friend struct shared_ptr_helper<StorageStripeLog>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
~StorageStripeLog() override;
|
||||||
|
|
||||||
String getName() const override { return "StripeLog"; }
|
String getName() const override { return "StripeLog"; }
|
||||||
|
|
||||||
Pipe read(
|
Pipe read(
|
||||||
@ -57,18 +62,36 @@ protected:
|
|||||||
size_t max_compress_block_size_);
|
size_t max_compress_block_size_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
struct ColumnData
|
using ReadLock = std::shared_lock<std::shared_timed_mutex>;
|
||||||
{
|
using WriteLock = std::unique_lock<std::shared_timed_mutex>;
|
||||||
String data_file_path;
|
|
||||||
};
|
|
||||||
using Files = std::map<String, ColumnData>; /// file name -> column data
|
|
||||||
|
|
||||||
DiskPtr disk;
|
/// Reads the index file if it hasn't read yet.
|
||||||
|
/// It is done lazily, so that with a large number of tables, the server starts quickly.
|
||||||
|
void loadIndices(std::chrono::seconds lock_timeout);
|
||||||
|
void loadIndices(const WriteLock &);
|
||||||
|
|
||||||
|
/// Saves the index file.
|
||||||
|
void saveIndices(const WriteLock &);
|
||||||
|
|
||||||
|
/// Removes all unsaved indices.
|
||||||
|
void removeUnsavedIndices(const WriteLock &);
|
||||||
|
|
||||||
|
/// Saves the sizes of the data and index files.
|
||||||
|
void saveFileSizes(const WriteLock &);
|
||||||
|
|
||||||
|
const DiskPtr disk;
|
||||||
String table_path;
|
String table_path;
|
||||||
|
|
||||||
size_t max_compress_block_size;
|
String data_file_path;
|
||||||
|
String index_file_path;
|
||||||
FileChecker file_checker;
|
FileChecker file_checker;
|
||||||
|
|
||||||
|
IndexForNativeFormat indices;
|
||||||
|
std::atomic<bool> indices_loaded = false;
|
||||||
|
size_t num_indices_saved = 0;
|
||||||
|
|
||||||
|
const size_t max_compress_block_size;
|
||||||
|
|
||||||
std::shared_timed_mutex rwlock;
|
std::shared_timed_mutex rwlock;
|
||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
Loading…
Reference in New Issue
Block a user