fixed nullptr error

This commit is contained in:
Nikita Vasilev 2019-01-07 21:53:51 +03:00
parent 6871665231
commit 69c6e77d53
6 changed files with 38 additions and 51 deletions

View File

@ -355,7 +355,9 @@ void MergeTreeData::setSkipIndexes(const ASTs & indexes_asts, bool only_check)
for (const auto &index_ast : indexes_asts) {
indexes.push_back(
std::move(MergeTreeIndexFactory::instance().get(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast))));
*this,
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast),
global_context)));
}
}
}

View File

@ -553,18 +553,23 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
Names all_column_names = data.getColumns().getNamesOfPhysical();
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
LOG_DEBUG(log, "Before extract");
NamesAndTypesList gathering_columns, merging_columns;
Names gathering_column_names, merging_column_names;
extractMergingAndGatheringColumns(
all_columns, data.sorting_key_expr, data.indexes,
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
LOG_DEBUG(log, "After extract");
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
data, future_part.name, future_part.part_info);
new_data_part->partition.assign(future_part.getPartition());
new_data_part->relative_path = TMP_PREFIX + future_part.name;
new_data_part->is_temp = true;
LOG_DEBUG(log, "New Part");
size_t sum_input_rows_upper_bound = merge_entry->total_size_marks * data.index_granularity;
MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate);

View File

@ -214,6 +214,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames());
MergedBlockOutputStream out(data, new_data_part->getFullPath(), columns, compression_codec);
for (auto index : data.indexes)
{
auto index_columns = index->expr->getRequiredColumnsWithTypes();
for (const auto & column : index_columns)
{
if (!block.has(column.name))
block.insert(ColumnWithTypeAndName(column.type, column.name));
}
index->expr->execute(block);
}
out.writePrefix();
out.writeWithPermutation(block, perm_ptr);
out.writeSuffixAndFinalizePart(new_data_part);

View File

@ -18,36 +18,6 @@ namespace ErrorCodes
}
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 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)
@ -55,7 +25,10 @@ void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creat
ErrorCodes::LOGICAL_ERROR);
}
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIndexDeclaration> node) const
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(
const MergeTreeData & data,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const
{
if (!node->type)
throw Exception(
@ -74,14 +47,7 @@ std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIn
}
}),
ErrorCodes::INCORRECT_QUERY);
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));
return it->second(data, node, context);
}
}

View File

@ -17,6 +17,7 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
namespace DB
{
class MergeTreeData;
class MergeTreeIndex;
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
@ -86,12 +87,7 @@ public:
};
class MergeTreeIndexes : public std::vector<MutableMergeTreeIndexPtr>
{
public:
void writeText(WriteBuffer & ostr) const;
void readText(ReadBuffer & istr);
};
using MergeTreeIndexes = std::vector<MutableMergeTreeIndexPtr>;
class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
@ -99,10 +95,16 @@ class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
friend class ext::singleton<MergeTreeIndexFactory>;
public:
using Creator = std::function<std::unique_ptr<MergeTreeIndex>(std::shared_ptr<ASTIndexDeclaration> node)>;
using Creator = std::function<
std::unique_ptr<MergeTreeIndex>(
const MergeTreeData & data,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)>;
std::unique_ptr<MergeTreeIndex> get(std::shared_ptr<ASTIndexDeclaration> node) const;
std::unique_ptr<MergeTreeIndex> get(const String & description) const;
std::unique_ptr<MergeTreeIndex> get(
const MergeTreeData & data,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const;
void registerIndex(const std::string & name, Creator creator);

View File

@ -75,9 +75,10 @@ public:
};
};
std::unique_ptr<MergeTreeIndex> MTItestCreator(std::shared_ptr<ASTIndexDeclaration> node) {
std::unique_ptr<MergeTreeIndex> MTItestCreator(
const MergeTreeData & data, std::shared_ptr<ASTIndexDeclaration> node, const Context & ) {
return std::make_unique<MergeTreeTestIndex>(
node->name, nullptr, node->granularity.get<size_t>(), Block{});
node->name, data.primary_key_expr, node->granularity.get<size_t>(), Block{});
}
}