提交 c99e7176 编写于 作者: A Alexey Milovidov 提交者: alexey-milovidov

Fixed bad code #2571

上级 6afec54f
......@@ -119,7 +119,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
if (!done_with_set)
{
if (!subquery.set->insertFromBlock(block, /*fill_set_elements=*/false))
if (!subquery.set->insertFromBlock(block))
done_with_set = true;
}
......
......@@ -1477,13 +1477,13 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
{
BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute();
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), true);
set->setHeader(res.in->getHeader());
while (Block block = res.in->read())
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
if (!set->insertFromBlock(block, true))
if (!set->insertFromBlock(block))
return;
}
......@@ -1589,7 +1589,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
return;
}
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), false);
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
......@@ -1711,8 +1711,8 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), create_ordered_set);
set->createFromAST(set_element_types, elements_ast, context);
prepared_sets[right_arg->range] = std::move(set);
}
......
......@@ -135,12 +135,22 @@ void Set::setHeader(const Block & block)
ConstNullMapPtr null_map{};
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
if (fill_set_elements)
{
/// Create empty columns with set values in advance.
/// It is needed because set may be empty, so method 'insertFromBlock' will be never called.
set_elements.reserve(keys_size);
for (const auto & type : data_types)
set_elements.emplace_back(removeNullable(type)->createColumn());
}
/// Choose data structure to use for the set.
data.init(data.chooseMethod(key_columns, key_sizes));
}
bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
bool Set::insertFromBlock(const Block & block)
{
std::unique_lock lock(rwlock);
......@@ -191,13 +201,10 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
if (fill_set_elements)
{
if (set_elements.empty())
set_elements.resize(keys_size);
for (size_t i = 0; i < keys_size; ++i)
{
auto filtered_column = block.getByPosition(i).column->filter(filter->getData(), rows);
if (!set_elements[i])
if (set_elements[i]->empty())
set_elements[i] = filtered_column;
else
set_elements[i]->assumeMutableRef().insertRangeFrom(*filtered_column, 0, filtered_column->size());
......@@ -224,7 +231,7 @@ static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const C
}
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements)
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context)
{
/// Will form a block with values from the set.
......@@ -295,7 +302,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
}
Block block = header.cloneWithColumns(std::move(columns));
insertFromBlock(block, fill_set_elements);
insertFromBlock(block);
}
......
......@@ -27,9 +27,13 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class Set
{
public:
Set(const SizeLimits & limits) :
log(&Logger::get("Set")),
limits(limits)
/// 'fill_set_elements': in addition to hash table
/// (that is useful only for checking that some value is in the set and may not store the original values),
/// store all set elements in explicit form.
/// This is needed for subsequent use for index.
Set(const SizeLimits & limits, bool fill_set_elements)
: log(&Logger::get("Set")),
limits(limits), fill_set_elements(fill_set_elements)
{
}
......@@ -43,7 +47,7 @@ public:
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
* 'fill_set_elements' - if true, fill vector of elements. For primary key to work.
*/
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements);
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context);
/** Create a Set from stream.
* Call setHeader, then call insertFromBlock for each block.
......@@ -51,7 +55,7 @@ public:
void setHeader(const Block & header);
/// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Block & block, bool fill_set_elements);
bool insertFromBlock(const Block & block);
/** For columns of 'block', check belonging of corresponding rows to the set.
* Return UInt8 column with the result.
......@@ -63,6 +67,7 @@ public:
const DataTypes & getDataTypes() const { return data_types; }
bool hasExplicitSetElements() const { return fill_set_elements; }
const Columns & getSetElements() const { return set_elements; }
private:
......@@ -96,6 +101,9 @@ private:
/// Limitations on the maximum size of the set
SizeLimits limits;
/// Do we need to additionally store all elements of the set in explicit form for subsequent use for index.
bool fill_set_elements;
/// If in the left part columns contains the same types as the elements of the set.
void executeOrdinary(
const ColumnRawPtrs & key_columns,
......@@ -164,8 +172,7 @@ class MergeTreeSetIndex
{
public:
/** Mapping for tuple positions from Set::set_elements to
* position of pk index and data type of this pk column
* and functions chain applied to this column.
* position of pk index and functions chain applied to this column.
*/
struct KeyTuplePositionMapping
{
......
......@@ -484,14 +484,17 @@ void KeyCondition::getKeyTuplePositionMapping(
}
/// Try to prepare KeyTuplePositionMapping for tuples from IN expression.
bool KeyCondition::isTupleIndexable(
bool KeyCondition::tryPrepareSetIndex(
const ASTPtr & node,
const Context & context,
RPNElement & out,
const SetPtr & prepared_set,
size_t & out_key_column_num)
{
/// The index can be prepared if the elements of the set were saved in advance.
if (!prepared_set->hasExplicitSetElements())
return false;
out_key_column_num = 0;
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
......@@ -641,7 +644,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
bool is_constant_transformed = false;
if (prepared_sets.count(args[1]->range)
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
&& tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
{
key_arg_pos = 0;
is_set_const = true;
......
......@@ -372,7 +372,10 @@ private:
const size_t tuple_index,
size_t & out_key_column_num);
bool isTupleIndexable(
/// If it's possible to make an RPNElement
/// that will filter values (possibly tuples) by the content of 'prepared_set',
/// do it and return true.
bool tryPrepareSetIndex(
const ASTPtr & node,
const Context & context,
RPNElement & out,
......
......@@ -105,7 +105,7 @@ StorageSet::StorageSet(
const String & name_,
const ColumnsDescription & columns_)
: StorageSetOrJoinBase{path_, name_, columns_},
set(std::make_shared<Set>(SizeLimits()))
set(std::make_shared<Set>(SizeLimits(), false))
{
Block header = getSampleBlock();
header = header.sortColumns();
......@@ -115,7 +115,7 @@ StorageSet::StorageSet(
}
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block, /*fill_set_elements=*/false); }
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); }
size_t StorageSet::getSize() const { return set->getTotalRowCount(); }
......@@ -129,7 +129,7 @@ void StorageSet::truncate(const ASTPtr &)
header = header.sortColumns();
increment = 0;
set = std::make_shared<Set>(SizeLimits());
set = std::make_shared<Set>(SizeLimits(), false);
set->setHeader(header);
};
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册