add rw for MergeTreeIndexes

This commit is contained in:
Nikita Vasilev 2019-01-05 21:33:30 +03:00
parent 82cc39d441
commit 61b9c7735b
3 changed files with 60 additions and 18 deletions

View File

@ -1,4 +1,9 @@
#include <Storages/MergeTree/MergeTreeIndexes.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
namespace DB
{
@ -11,18 +16,45 @@ namespace ErrorCodes
}
IndexType IndexCondition::indexType() const {
return index->indexType();
void MergeTreeIndexes::writeText(DB::WriteBuffer &ostr) const
{
writeString("indexes format version: 1\n", ostr);
DB::writeText(size(), ostr);
writeString(" indexes:\n", ostr);
for (auto index : *this) {
index->writeText(ostr);
writeChar('\n', ostr);
}
}
void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creator) {
void MergeTreeIndexes::readText(DB::ReadBuffer &istr)
{
const MergeTreeIndexFactory & factory = MergeTreeIndexFactory::instance();
assertString("indexes format version: 1\n", istr);
size_t count;
DB::readText(count, istr);
assertString(" indexes:\n", istr);
reserve(count);
for (size_t i = 0; i < count; ++i) {
String index_descr;
readString(index_descr, istr);
emplace_back(factory.get(index_descr));
assertChar('\n', istr);
}
}
void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creator)
{
if (!indexes.emplace(name, std::move(creator)).second)
throw Exception("MergeTreeIndexFactory: the Index creator name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIndexDeclaration> node) const {
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIndexDeclaration> node) const
{
if (!node->type)
throw Exception(
"for INDEX TYPE is required",
@ -35,4 +67,11 @@ std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIn
return it->second(node);
}
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(const String & description) const
{
ParserIndexDeclaration parser;
ASTPtr ast = parseQuery(parser, description.data(), description.data() + description.size(), "index factory", 0);
return get(std::dynamic_pointer_cast<ASTIndexDeclaration>(ast));
}
}

View File

@ -17,16 +17,11 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
namespace DB
{
enum class IndexType {
NONE = 0
};
class MergeTreeIndex;
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<MergeTreeIndex>;
using MergeTreeIndexes = std::vector<MutableMergeTreeIndexPtr>;
/// Condition on the index.
@ -37,8 +32,6 @@ class IndexCondition {
public:
virtual ~IndexCondition() = default;
IndexType indexType() const;
/// Checks if this index is useful for query.
virtual bool alwaysUnknownOrTrue() const = 0;
@ -82,18 +75,18 @@ public:
virtual ~MergeTreeIndex() {};
virtual IndexType indexType() const = 0;
virtual String indexType() const { return "UNKNOWN"; };
/// gets filename without extension
virtual String getFileName() const = 0;
String getFileExt() const { return ".idx"; };
virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0;
virtual IndexConditionPtr createIndexConditionOnPart(
const SelectQueryInfo & query_info, const Context & context) const = 0;
virtual void writeText(WriteBuffer & ostr) const = 0;
String name;
ExpressionActionsPtr expr;
size_t granularity;
@ -103,6 +96,13 @@ public:
};
class MergeTreeIndexes : public std::vector<MutableMergeTreeIndexPtr>
{
void writeText(WriteBuffer & ostr) const;
void readText(ReadBuffer & istr);
};
class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
{
friend class ext::singleton<MergeTreeIndexFactory>;
@ -111,6 +111,7 @@ public:
using Creator = std::function<std::unique_ptr<MergeTreeIndex>(std::shared_ptr<ASTIndexDeclaration> node)>;
std::unique_ptr<MergeTreeIndex> get(std::shared_ptr<ASTIndexDeclaration> node) const;
std::unique_ptr<MergeTreeIndex> get(const String & description) const;
void registerIndex(const std::string & name, Creator creator);

View File

@ -30,12 +30,13 @@ namespace ErrorCodes
namespace
{
/** To read and checksum single stream (a pair of .bin, .mrk files) for a single column.
/** To read and checksum single stream (a pair of .bin, .mrk files) for a single column or secondary index.
*/
class Stream
{
public:
String base_name;
String bin_file_ext;
String bin_file_path;
String mrk_file_path;
private:
@ -50,10 +51,11 @@ private:
public:
HashingReadBuffer mrk_hashing_buf;
Stream(const String & path, const String & base_name)
Stream(const String & path, const String & base_name, const String & bin_file_ext = ".bin")
:
base_name(base_name),
bin_file_path(path + base_name + ".bin"),
bin_file_ext(bin_file_ext),
bin_file_path(path + base_name + bin_file_ext),
mrk_file_path(path + base_name + ".mrk"),
file_buf(bin_file_path),
compressed_hashing_buf(file_buf),
@ -118,7 +120,7 @@ public:
void saveChecksums(MergeTreeData::DataPart::Checksums & checksums)
{
checksums.files[base_name + ".bin"] = MergeTreeData::DataPart::Checksums::Checksum(
checksums.files[base_name + bin_file_ext] = MergeTreeData::DataPart::Checksums::Checksum(
compressed_hashing_buf.count(), compressed_hashing_buf.getHash(),
uncompressed_hashing_buf.count(), uncompressed_hashing_buf.getHash());