mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +00:00
fixed nullptr error
This commit is contained in:
parent
6871665231
commit
69c6e77d53
@ -355,7 +355,9 @@ void MergeTreeData::setSkipIndexes(const ASTs & indexes_asts, bool only_check)
|
|||||||
for (const auto &index_ast : indexes_asts) {
|
for (const auto &index_ast : indexes_asts) {
|
||||||
indexes.push_back(
|
indexes.push_back(
|
||||||
std::move(MergeTreeIndexFactory::instance().get(
|
std::move(MergeTreeIndexFactory::instance().get(
|
||||||
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast))));
|
*this,
|
||||||
|
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast),
|
||||||
|
global_context)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -553,18 +553,23 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
|||||||
Names all_column_names = data.getColumns().getNamesOfPhysical();
|
Names all_column_names = data.getColumns().getNamesOfPhysical();
|
||||||
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
|
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Before extract");
|
||||||
NamesAndTypesList gathering_columns, merging_columns;
|
NamesAndTypesList gathering_columns, merging_columns;
|
||||||
Names gathering_column_names, merging_column_names;
|
Names gathering_column_names, merging_column_names;
|
||||||
extractMergingAndGatheringColumns(
|
extractMergingAndGatheringColumns(
|
||||||
all_columns, data.sorting_key_expr, data.indexes,
|
all_columns, data.sorting_key_expr, data.indexes,
|
||||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
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>(
|
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
|
||||||
data, future_part.name, future_part.part_info);
|
data, future_part.name, future_part.part_info);
|
||||||
new_data_part->partition.assign(future_part.getPartition());
|
new_data_part->partition.assign(future_part.getPartition());
|
||||||
new_data_part->relative_path = TMP_PREFIX + future_part.name;
|
new_data_part->relative_path = TMP_PREFIX + future_part.name;
|
||||||
new_data_part->is_temp = true;
|
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;
|
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);
|
MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate);
|
||||||
|
@ -214,6 +214,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
|||||||
NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames());
|
NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames());
|
||||||
MergedBlockOutputStream out(data, new_data_part->getFullPath(), columns, compression_codec);
|
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.writePrefix();
|
||||||
out.writeWithPermutation(block, perm_ptr);
|
out.writeWithPermutation(block, perm_ptr);
|
||||||
out.writeSuffixAndFinalizePart(new_data_part);
|
out.writeSuffixAndFinalizePart(new_data_part);
|
||||||
|
@ -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)
|
void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creator)
|
||||||
{
|
{
|
||||||
if (!indexes.emplace(name, std::move(creator)).second)
|
if (!indexes.emplace(name, std::move(creator)).second)
|
||||||
@ -55,7 +25,10 @@ void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creat
|
|||||||
ErrorCodes::LOGICAL_ERROR);
|
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)
|
if (!node->type)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
@ -74,14 +47,7 @@ std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(std::shared_ptr<ASTIn
|
|||||||
}
|
}
|
||||||
}),
|
}),
|
||||||
ErrorCodes::INCORRECT_QUERY);
|
ErrorCodes::INCORRECT_QUERY);
|
||||||
return it->second(node);
|
return it->second(data, node, context);
|
||||||
}
|
|
||||||
|
|
||||||
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));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -17,6 +17,7 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class MergeTreeData;
|
||||||
class MergeTreeIndex;
|
class MergeTreeIndex;
|
||||||
|
|
||||||
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
|
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
|
||||||
@ -86,12 +87,7 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
class MergeTreeIndexes : public std::vector<MutableMergeTreeIndexPtr>
|
using MergeTreeIndexes = std::vector<MutableMergeTreeIndexPtr>;
|
||||||
{
|
|
||||||
public:
|
|
||||||
void writeText(WriteBuffer & ostr) const;
|
|
||||||
void readText(ReadBuffer & istr);
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
|
class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
|
||||||
@ -99,10 +95,16 @@ class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
|
|||||||
friend class ext::singleton<MergeTreeIndexFactory>;
|
friend class ext::singleton<MergeTreeIndexFactory>;
|
||||||
|
|
||||||
public:
|
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(
|
||||||
std::unique_ptr<MergeTreeIndex> get(const String & description) const;
|
const MergeTreeData & data,
|
||||||
|
std::shared_ptr<ASTIndexDeclaration> node,
|
||||||
|
const Context & context) const;
|
||||||
|
|
||||||
void registerIndex(const std::string & name, Creator creator);
|
void registerIndex(const std::string & name, Creator creator);
|
||||||
|
|
||||||
|
@ -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>(
|
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{});
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
Loading…
Reference in New Issue
Block a user