未验证 提交 b888c867 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #10026 from CurtizJ/speedup-index

Improve performance of index analysis with monotonic functions [2].
...@@ -596,6 +596,14 @@ BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges, ...@@ -596,6 +596,14 @@ BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges,
}; };
} }
bool MergeTreeSetIndex::hasMonotonicFunctionsChain() const
{
for (const auto & mapping : indexes_mapping)
if (!mapping.functions.empty())
return true;
return false;
}
void ValueWithInfinity::update(const Field & x) void ValueWithInfinity::update(const Field & x)
{ {
/// Keep at most one element in column. /// Keep at most one element in column.
...@@ -607,8 +615,11 @@ void ValueWithInfinity::update(const Field & x) ...@@ -607,8 +615,11 @@ void ValueWithInfinity::update(const Field & x)
const IColumn & ValueWithInfinity::getColumnIfFinite() const const IColumn & ValueWithInfinity::getColumnIfFinite() const
{ {
#ifndef NDEBUG
if (type != NORMAL) if (type != NORMAL)
throw Exception("Trying to get column of infinite type", ErrorCodes::LOGICAL_ERROR); throw Exception("Trying to get column of infinite type", ErrorCodes::LOGICAL_ERROR);
#endif
return *column; return *column;
} }
......
...@@ -231,6 +231,8 @@ public: ...@@ -231,6 +231,8 @@ public:
size_t size() const { return ordered_set.at(0)->size(); } size_t size() const { return ordered_set.at(0)->size(); }
bool hasMonotonicFunctionsChain() const;
BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types); BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types);
private: private:
......
...@@ -98,8 +98,8 @@ void IMergeTreeDataPart::MinMaxIndex::update(const Block & block, const Names & ...@@ -98,8 +98,8 @@ void IMergeTreeDataPart::MinMaxIndex::update(const Block & block, const Names &
for (size_t i = 0; i < column_names.size(); ++i) for (size_t i = 0; i < column_names.size(); ++i)
{ {
Field min_value; FieldRef min_value;
Field max_value; FieldRef max_value;
const ColumnWithTypeAndName & column = block.getByName(column_names[i]); const ColumnWithTypeAndName & column = block.getByName(column_names[i]);
column.column->getExtremes(min_value, max_value); column.column->getExtremes(min_value, max_value);
......
...@@ -338,44 +338,6 @@ inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVi ...@@ -338,44 +338,6 @@ inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVi
inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); }
FieldWithInfinity::FieldWithInfinity(const Field & field_)
: field(field_),
type(Type::NORMAL)
{
}
FieldWithInfinity::FieldWithInfinity(Field && field_)
: field(std::move(field_)),
type(Type::NORMAL)
{
}
FieldWithInfinity::FieldWithInfinity(const Type type_)
: type(type_)
{
}
FieldWithInfinity FieldWithInfinity::getMinusInfinity()
{
return FieldWithInfinity(Type::MINUS_INFINITY);
}
FieldWithInfinity FieldWithInfinity::getPlusInfinity()
{
return FieldWithInfinity(Type::PLUS_INFINITY);
}
bool FieldWithInfinity::operator<(const FieldWithInfinity & other) const
{
return type < other.type || (type == other.type && type == Type::NORMAL && field < other.field);
}
bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const
{
return type == other.type && (type != Type::NORMAL || field == other.field);
}
/** Calculate expressions, that depend only on constants. /** Calculate expressions, that depend only on constants.
* For index to work when something like "WHERE Date = toDate(now())" is written. * For index to work when something like "WHERE Date = toDate(now())" is written.
*/ */
...@@ -480,24 +442,41 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants ...@@ -480,24 +442,41 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants
} }
static void applyFunction( static Field applyFunctionForField(
const FunctionBasePtr & func, const FunctionBasePtr & func,
const DataTypePtr & arg_type, const Field & arg_value, const DataTypePtr & arg_type,
DataTypePtr & res_type, Field & res_value) const Field & arg_value)
{ {
res_type = func->getReturnType();
Block block Block block
{ {
{ arg_type->createColumnConst(1, arg_value), arg_type, "x" }, { arg_type->createColumnConst(1, arg_value), arg_type, "x" },
{ nullptr, res_type, "y" } { nullptr, func->getReturnType(), "y" }
}; };
func->execute(block, {0}, 1, 1); func->execute(block, {0}, 1, 1);
return (*block.safeGetByPosition(1).column)[0];
block.safeGetByPosition(1).column->get(0, res_value);
} }
static FieldRef applyFunction(FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
{
/// Fallback for fields without block reference.
if (field.isExplicit())
return applyFunctionForField(func, current_type, field);
String result_name = "_" + func->getName() + "_" + toString(field.column_idx);
size_t result_idx;
const auto & block = field.block;
if (!block->has(result_name))
{
result_idx = block->columns();
field.block->insert({nullptr, func->getReturnType(), result_name});
func->execute(*block, {field.column_idx}, result_idx, block->rows());
}
else
result_idx = block->getPositionByName(result_name);
return {field.block, field.row_idx, result_idx};
}
void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants) void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants)
{ {
...@@ -569,12 +548,8 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( ...@@ -569,12 +548,8 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
return false; return false;
// Apply the next transformation step // Apply the next transformation step
DataTypePtr new_type; out_value = applyFunctionForField(a.function_base, out_type, out_value);
applyFunction(a.function_base, out_type, out_value, new_type, out_value); out_type = a.function_base->getReturnType();
if (!new_type)
return false;
out_type.swap(new_type);
expr_name = a.result_name; expr_name = a.result_name;
// Transformation results in a key expression, accept // Transformation results in a key expression, accept
...@@ -957,8 +932,8 @@ String KeyCondition::toString() const ...@@ -957,8 +932,8 @@ String KeyCondition::toString() const
template <typename F> template <typename F>
static BoolMask forAnyHyperrectangle( static BoolMask forAnyHyperrectangle(
size_t key_size, size_t key_size,
const Field * key_left, const FieldRef * key_left,
const Field * key_right, const FieldRef * key_right,
bool left_bounded, bool left_bounded,
bool right_bounded, bool right_bounded,
std::vector<Range> & hyperrectangle, std::vector<Range> & hyperrectangle,
...@@ -1049,8 +1024,8 @@ static BoolMask forAnyHyperrectangle( ...@@ -1049,8 +1024,8 @@ static BoolMask forAnyHyperrectangle(
BoolMask KeyCondition::checkInRange( BoolMask KeyCondition::checkInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef * right_key,
const DataTypes & data_types, const DataTypes & data_types,
bool right_bounded, bool right_bounded,
BoolMask initial_mask) const BoolMask initial_mask) const
...@@ -1102,19 +1077,12 @@ std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange( ...@@ -1102,19 +1077,12 @@ std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
return {}; return {};
} }
/// Apply the function.
DataTypePtr new_type;
if (!key_range.left.isNull()) if (!key_range.left.isNull())
applyFunction(func, current_type, key_range.left, new_type, key_range.left); key_range.left = applyFunction(func, current_type, key_range.left);
if (!key_range.right.isNull()) if (!key_range.right.isNull())
applyFunction(func, current_type, key_range.right, new_type, key_range.right); key_range.right = applyFunction(func, current_type, key_range.right);
if (!new_type) current_type = func->getReturnType();
{
return {};
}
current_type.swap(new_type);
if (!monotonicity.is_positive) if (!monotonicity.is_positive)
key_range.swapLeftAndRight(); key_range.swapLeftAndRight();
...@@ -1220,8 +1188,8 @@ BoolMask KeyCondition::checkInHyperrectangle( ...@@ -1220,8 +1188,8 @@ BoolMask KeyCondition::checkInHyperrectangle(
BoolMask KeyCondition::checkInRange( BoolMask KeyCondition::checkInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef * right_key,
const DataTypes & data_types, const DataTypes & data_types,
BoolMask initial_mask) const BoolMask initial_mask) const
{ {
...@@ -1231,8 +1199,8 @@ BoolMask KeyCondition::checkInRange( ...@@ -1231,8 +1199,8 @@ BoolMask KeyCondition::checkInRange(
bool KeyCondition::mayBeTrueInRange( bool KeyCondition::mayBeTrueInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef * right_key,
const DataTypes & data_types) const const DataTypes & data_types) const
{ {
return checkInRange(used_key_size, left_key, right_key, data_types, true, BoolMask::consider_only_can_be_true).can_be_true; return checkInRange(used_key_size, left_key, right_key, data_types, true, BoolMask::consider_only_can_be_true).can_be_true;
...@@ -1241,7 +1209,7 @@ bool KeyCondition::mayBeTrueInRange( ...@@ -1241,7 +1209,7 @@ bool KeyCondition::mayBeTrueInRange(
BoolMask KeyCondition::checkAfter( BoolMask KeyCondition::checkAfter(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const DataTypes & data_types, const DataTypes & data_types,
BoolMask initial_mask) const BoolMask initial_mask) const
{ {
...@@ -1251,7 +1219,7 @@ BoolMask KeyCondition::checkAfter( ...@@ -1251,7 +1219,7 @@ BoolMask KeyCondition::checkAfter(
bool KeyCondition::mayBeTrueAfter( bool KeyCondition::mayBeTrueAfter(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const DataTypes & data_types) const const DataTypes & data_types) const
{ {
return checkInRange(used_key_size, left_key, nullptr, data_types, false, BoolMask::consider_only_can_be_true).can_be_true; return checkInRange(used_key_size, left_key, nullptr, data_types, false, BoolMask::consider_only_can_be_true).can_be_true;
...@@ -1382,4 +1350,13 @@ size_t KeyCondition::getMaxKeyColumn() const ...@@ -1382,4 +1350,13 @@ size_t KeyCondition::getMaxKeyColumn() const
return res; return res;
} }
bool KeyCondition::hasMonotonicFunctionsChain() const
{
for (const auto & element : rpn)
if (!element.monotonic_functions_chain.empty()
|| (element.set_index && element.set_index->hasMonotonicFunctionsChain()))
return true;
return false;
}
} }
...@@ -15,10 +15,6 @@ ...@@ -15,10 +15,6 @@
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
}
class IFunction; class IFunction;
using FunctionBasePtr = std::shared_ptr<IFunctionBase>; using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
...@@ -26,6 +22,33 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>; ...@@ -26,6 +22,33 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class ExpressionActions; class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>; using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/** A field, that can be stored in two reperesenations:
* - A standalone field.
* - A field with reference to its position in a block.
* It's needed for execution of functions on ranges during
* index analysis. If function was executed once for field,
* its result would be cached for whole block for which field's reference points to.
*/
struct FieldRef : public Field
{
FieldRef() = default;
/// Create as explicit field without block.
template <typename T>
FieldRef(T && value) : Field(std::forward<T>(value)) {}
/// Create as reference to field in block.
FieldRef(Block * block_, size_t row_idx_, size_t column_idx_)
: Field((*block_->getByPosition(column_idx_).column)[row_idx_]),
block(block_), row_idx(row_idx_), column_idx(column_idx_) {}
bool isExplicit() const { return block == nullptr; }
Block * block = nullptr;
size_t row_idx = 0;
size_t column_idx = 0;
};
/** Range with open or closed ends; possibly unbounded. /** Range with open or closed ends; possibly unbounded.
*/ */
struct Range struct Range
...@@ -35,8 +58,8 @@ private: ...@@ -35,8 +58,8 @@ private:
static bool less(const Field & lhs, const Field & rhs); static bool less(const Field & lhs, const Field & rhs);
public: public:
Field left; /// the left border, if any FieldRef left; /// the left border, if any
Field right; /// the right border, if any FieldRef right; /// the right border, if any
bool left_bounded = false; /// bounded at the left bool left_bounded = false; /// bounded at the left
bool right_bounded = false; /// bounded at the right bool right_bounded = false; /// bounded at the right
bool left_included = false; /// includes the left border, if any bool left_included = false; /// includes the left border, if any
...@@ -46,11 +69,11 @@ public: ...@@ -46,11 +69,11 @@ public:
Range() {} Range() {}
/// One point. /// One point.
Range(const Field & point) Range(const FieldRef & point)
: left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {} : left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {}
/// A bounded two-sided range. /// A bounded two-sided range.
Range(const Field & left_, bool left_included_, const Field & right_, bool right_included_) Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_)
: left(left_), right(right_), : left(left_), right(right_),
left_bounded(true), right_bounded(true), left_bounded(true), right_bounded(true),
left_included(left_included_), right_included(right_included_) left_included(left_included_), right_included(right_included_)
...@@ -58,7 +81,7 @@ public: ...@@ -58,7 +81,7 @@ public:
shrinkToIncludedIfPossible(); shrinkToIncludedIfPossible();
} }
static Range createRightBounded(const Field & right_point, bool right_included) static Range createRightBounded(const FieldRef & right_point, bool right_included)
{ {
Range r; Range r;
r.right = right_point; r.right = right_point;
...@@ -68,7 +91,7 @@ public: ...@@ -68,7 +91,7 @@ public:
return r; return r;
} }
static Range createLeftBounded(const Field & left_point, bool left_included) static Range createLeftBounded(const FieldRef & left_point, bool left_included)
{ {
Range r; Range r;
r.left = left_point; r.left = left_point;
...@@ -84,7 +107,7 @@ public: ...@@ -84,7 +107,7 @@ public:
*/ */
void shrinkToIncludedIfPossible() void shrinkToIncludedIfPossible()
{ {
if (left_bounded && !left_included) if (left.isExplicit() && left_bounded && !left_included)
{ {
if (left.getType() == Field::Types::UInt64 && left.get<UInt64>() != std::numeric_limits<UInt64>::max()) if (left.getType() == Field::Types::UInt64 && left.get<UInt64>() != std::numeric_limits<UInt64>::max())
{ {
...@@ -97,7 +120,7 @@ public: ...@@ -97,7 +120,7 @@ public:
left_included = true; left_included = true;
} }
} }
if (right_bounded && !right_included) if (right.isExplicit() && right_bounded && !right_included)
{ {
if (right.getType() == Field::Types::UInt64 && right.get<UInt64>() != std::numeric_limits<UInt64>::min()) if (right.getType() == Field::Types::UInt64 && right.get<UInt64>() != std::numeric_limits<UInt64>::min())
{ {
...@@ -120,13 +143,13 @@ public: ...@@ -120,13 +143,13 @@ public:
} }
/// x contained in the range /// x contained in the range
bool contains(const Field & x) const bool contains(const FieldRef & x) const
{ {
return !leftThan(x) && !rightThan(x); return !leftThan(x) && !rightThan(x);
} }
/// x is to the left /// x is to the left
bool rightThan(const Field & x) const bool rightThan(const FieldRef & x) const
{ {
return (left_bounded return (left_bounded
? !(less(left, x) || (left_included && equals(x, left))) ? !(less(left, x) || (left_included && equals(x, left)))
...@@ -134,7 +157,7 @@ public: ...@@ -134,7 +157,7 @@ public:
} }
/// x is to the right /// x is to the right
bool leftThan(const Field & x) const bool leftThan(const FieldRef & x) const
{ {
return (right_bounded return (right_bounded
? !(less(x, right) || (right_included && equals(x, right))) ? !(less(x, right) || (right_included && equals(x, right)))
...@@ -195,42 +218,6 @@ public: ...@@ -195,42 +218,6 @@ public:
String toString() const; String toString() const;
}; };
/// Class that extends arbitrary objects with infinities, like +-inf for floats
class FieldWithInfinity
{
public:
enum Type
{
MINUS_INFINITY = -1,
NORMAL = 0,
PLUS_INFINITY = 1
};
explicit FieldWithInfinity(const Field & field_);
FieldWithInfinity(Field && field_);
static FieldWithInfinity getMinusInfinity();
static FieldWithInfinity getPlusInfinity();
bool operator<(const FieldWithInfinity & other) const;
bool operator==(const FieldWithInfinity & other) const;
Field getFieldIfFinite() const
{
if (type != NORMAL)
throw Exception("Trying to get field of infinite type", ErrorCodes::BAD_TYPE_OF_FIELD);
return field;
}
private:
Field field;
Type type;
FieldWithInfinity(const Type type_);
};
/** Condition on the index. /** Condition on the index.
* *
* Consists of the conditions for the key belonging to all possible ranges or sets, * Consists of the conditions for the key belonging to all possible ranges or sets,
...@@ -261,8 +248,8 @@ public: ...@@ -261,8 +248,8 @@ public:
/// one of the resulting mask components (see BoolMask::consider_only_can_be_XXX). /// one of the resulting mask components (see BoolMask::consider_only_can_be_XXX).
BoolMask checkInRange( BoolMask checkInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef* right_key,
const DataTypes & data_types, const DataTypes & data_types,
BoolMask initial_mask = BoolMask(false, false)) const; BoolMask initial_mask = BoolMask(false, false)) const;
...@@ -270,7 +257,7 @@ public: ...@@ -270,7 +257,7 @@ public:
/// left_key must contain all the fields in the sort_descr in the appropriate order. /// left_key must contain all the fields in the sort_descr in the appropriate order.
BoolMask checkAfter( BoolMask checkAfter(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const DataTypes & data_types, const DataTypes & data_types,
BoolMask initial_mask = BoolMask(false, false)) const; BoolMask initial_mask = BoolMask(false, false)) const;
...@@ -278,15 +265,15 @@ public: ...@@ -278,15 +265,15 @@ public:
/// This is more efficient than checkInRange(...).can_be_true. /// This is more efficient than checkInRange(...).can_be_true.
bool mayBeTrueInRange( bool mayBeTrueInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef * right_key,
const DataTypes & data_types) const; const DataTypes & data_types) const;
/// Same as checkAfter, but calculate only may_be_true component of a result. /// Same as checkAfter, but calculate only may_be_true component of a result.
/// This is more efficient than checkAfter(...).can_be_true. /// This is more efficient than checkAfter(...).can_be_true.
bool mayBeTrueAfter( bool mayBeTrueAfter(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const DataTypes & data_types) const; const DataTypes & data_types) const;
/// Checks that the index can not be used. /// Checks that the index can not be used.
...@@ -295,6 +282,8 @@ public: ...@@ -295,6 +282,8 @@ public:
/// Get the maximum number of the key element used in the condition. /// Get the maximum number of the key element used in the condition.
size_t getMaxKeyColumn() const; size_t getMaxKeyColumn() const;
bool hasMonotonicFunctionsChain() const;
/// Impose an additional condition: the value in the column `column` must be in the range `range`. /// Impose an additional condition: the value in the column `column` must be in the range `range`.
/// Returns whether there is such a column in the key. /// Returns whether there is such a column in the key.
bool addCondition(const String & column, const Range & range); bool addCondition(const String & column, const Range & range);
...@@ -374,8 +363,8 @@ public: ...@@ -374,8 +363,8 @@ public:
private: private:
BoolMask checkInRange( BoolMask checkInRange(
size_t used_key_size, size_t used_key_size,
const Field * left_key, const FieldRef * left_key,
const Field * right_key, const FieldRef * right_key,
const DataTypes & data_types, const DataTypes & data_types,
bool right_bounded, bool right_bounded,
BoolMask initial_mask) const; BoolMask initial_mask) const;
......
...@@ -1201,11 +1201,33 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( ...@@ -1201,11 +1201,33 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
* If fits, split it into smaller ones and put them on the stack. If not, discard it. * If fits, split it into smaller ones and put them on the stack. If not, discard it.
* If the segment is already of one mark length, add it to response and discard it. * If the segment is already of one mark length, add it to response and discard it.
*/ */
std::vector<MarkRange> ranges_stack{ {0, marks_count} }; std::vector<MarkRange> ranges_stack = { {0, marks_count} };
std::function<void(size_t, size_t, FieldRef &)> create_field_ref;
/// If there are no monotonic functions, there is no need to save block reference.
/// Passing explicit field to FieldRef allows to optimize ranges and shows better performance.
if (key_condition.hasMonotonicFunctionsChain())
{
auto index_block = std::make_shared<Block>();
for (size_t i = 0; i < used_key_size; ++i)
index_block->insert({index[i], data.primary_key_data_types[i], data.primary_key_columns[i]});
create_field_ref = [index_block](size_t row, size_t column, FieldRef & field)
{
field = {index_block.get(), row, column};
};
}
else
{
create_field_ref = [&index](size_t row, size_t column, FieldRef & field)
{
index[column]->get(row, field);
};
}
/// NOTE Creating temporary Field objects to pass to KeyCondition. /// NOTE Creating temporary Field objects to pass to KeyCondition.
Row index_left(used_key_size); std::vector<FieldRef> index_left(used_key_size);
Row index_right(used_key_size); std::vector<FieldRef> index_right(used_key_size);
while (!ranges_stack.empty()) while (!ranges_stack.empty())
{ {
...@@ -1216,7 +1238,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( ...@@ -1216,7 +1238,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
if (range.end == marks_count && !has_final_mark) if (range.end == marks_count && !has_final_mark)
{ {
for (size_t i = 0; i < used_key_size; ++i) for (size_t i = 0; i < used_key_size; ++i)
index[i]->get(range.begin, index_left[i]); create_field_ref(range.begin, i, index_left[i]);
may_be_true = key_condition.mayBeTrueAfter( may_be_true = key_condition.mayBeTrueAfter(
used_key_size, index_left.data(), data.primary_key_data_types); used_key_size, index_left.data(), data.primary_key_data_types);
...@@ -1228,8 +1250,8 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( ...@@ -1228,8 +1250,8 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
for (size_t i = 0; i < used_key_size; ++i) for (size_t i = 0; i < used_key_size; ++i)
{ {
index[i]->get(range.begin, index_left[i]); create_field_ref(range.begin, i, index_left[i]);
index[i]->get(range.end, index_right[i]); create_field_ref(range.end, i, index_right[i]);
} }
may_be_true = key_condition.mayBeTrueInRange( may_be_true = key_condition.mayBeTrueInRange(
...@@ -1254,9 +1276,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( ...@@ -1254,9 +1276,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
size_t end; size_t end;
for (end = range.end; end > range.begin + step; end -= step) for (end = range.end; end > range.begin + step; end -= step)
ranges_stack.push_back(MarkRange(end - step, end)); ranges_stack.emplace_back(end - step, end);
ranges_stack.push_back(MarkRange(range.begin, end)); ranges_stack.emplace_back(range.begin, end);
} }
} }
} }
......
...@@ -100,8 +100,8 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s ...@@ -100,8 +100,8 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
size_t rows_read = std::min(limit, block.rows() - *pos); size_t rows_read = std::min(limit, block.rows() - *pos);
Field field_min; FieldRef field_min;
Field field_max; FieldRef field_max;
for (size_t i = 0; i < index.columns.size(); ++i) for (size_t i = 0; i < index.columns.size(); ++i)
{ {
const auto & column = block.getByName(index.columns[i]).column; const auto & column = block.getByName(index.columns[i]).column;
......
...@@ -14,7 +14,14 @@ ...@@ -14,7 +14,14 @@
</stop_conditions> </stop_conditions>
<!-- IN is used at index analysis --> <!-- IN is used at index analysis -->
<query>SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM zeros(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break'</query> <query>SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break'</query>
<query>SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break'</query>
<!-- Analyze only one range -->
<query>SELECT count() FROM test_in WHERE -toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break'</query>
<!-- Analyze all ranges -->
<query>SELECT count() FROM test_in WHERE -toInt64(a) NOT IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break'</query>
<!-- Test with explicitly defined large set (10000 elemnets) --> <!-- Test with explicitly defined large set (10000 elemnets) -->
<query>SELECT count() FROM numbers(1000) WHERE toString(number) IN ('41577', '83972', '51697', '50014', '37553', '93459', '87438', '95971', '83186', '74326', '67871', '50406', '83678', '29655', '18580', '83905', '61518', '29059', '56700', '82787', '98672', '30884', '81822', '39850', '80852', '57627', '91346', '64522', '17781', '49467', '41099', '41929', '85618', '91389', '68564', '91769', '81219', '52218', '37220', '97097', '2129', '9886', '52049', '34847', '25364', '36429', '76897', '71868', '58121', '71199', '84819', '69991', '34046', '64507', '34892', '24228', '36986', '28588', '51159', '53444', '80531', '9941', '20256', '48103', '32565', '62890', '5379', '60302', '46434', '3205', '18821', '31030', '19794', '71557', '71703', '15024', '14004', '82164', '95659', '40227', '83358', '24395', '9610', '19814', '48491', '66412', '16012', '71586', '42143', '51103', '24463', '89949', '35694', '39193', '63904', '40489', '77144', '94014', '84836', '9980', '46554', '43905', '25588', '25205', '72624', '10249', '35888', '98478', '99030', '26834', '31', '81499', '14847', '82997', '92357', '92893', '17426', '56630', '22252', '68119', '62710', '8740', '82144', '79916', '23391', '30192', '99271', '96435', '44237', '98327', '69481', '16691', '13643', '84554', '38571', '70926', '99283', '79000', '20926', '86495', '4834', '1222', '39486', '57697', '58002', '40790', '15623', '3999', '31515', '12694', '26143', '35951', '54085', '97534', '35329', '73535', '88715', '29572', '75799', '45166', '32066', '48023', '69523', '93150', '8740', '96790', '15534', '63252', '5142', '67045', '93992', '16663', '292', '63924', '6588', '12190', '31506', '69590', '35394', '55168', '65223', '79183', '32600', '69676', '28316', '72111', '53531', '15073', '41127', '73451', '24725', '61647', '65315', '41143', '26493', '95608', '34407', '76098', '53105', '83691', '48755', '35696', '62587', '81826', '3963', '45766', '82751', '12430', '97685', '29919', '78155', '71636', '50215', '89734', '9892', '47151', '54855', '3428', '9712', '52592', '2403', '79602', '81243', '79859', '57361', '82000', '42107', '28860', '99591', '28296', '57337', '64969', '32332', '25535', '30924', '21313', '32554', '17342', '87311', '19825', '24898', '61323', '83209', '79322', '79009', '50746', '33396', '62033', '16548', '17427', '24073', '34640', '52368', '4724', '80408', '40', '33787', '16666', '19665', '86751', '27264', '2241', '88134', '53566', '10589', '79711', '92823', '58972', '91767', '60885', '51659', '7867', '96849', '30360', '20914', '9584', '1250', '22871', '23282', '99312', '4683', '33429', '68361', '82614', '81440', '47863', '69790', '11968', '75210', '66854', '37002', '61142', '71514', '1588', '42336', '11069', '26291', '2261', '71056', '13492', '9133', '91216', '72207', '71586', '86535', '83898', '24392', '45384', '48545', '61972', '503', '80180', '35834', '97025', '70411', '55039', '35430', '27631', '82533', '96831', '74077', '42533', '14451', '26943', '53783', '69489', '71969', '8432', '37230', '61348', '19472', '59115', '9886', '50951', '57109', '7141', '1902', '84130', '4323', '55889', '47784', '2220', '75988', '66988', '63721', '8131', '95601', '95207', '2311', '26541', '50991', '6717', '2969', '71857', '51034', '65958', '94716', '90275', '21012', '46859', '7984', '31131', '46457', '69578', '44540', '7294', '80117', '9925', '60155', '90608', '82684', '32193', '87071', '28006', '87604', '24501', '79087', '2848', '29237', '11221', '81319', '40966', '87641', '35325', '78705', '88636', '78717', '62831', '56390', '99271', '43821', '14453', '17923', '62695', '77322', '21038', '67677', '41271', '4376', '65426', '46091', '19887', '97251', '55583', '58763', '3826', '35037', '73533', '64267', '82319', '9836', '42622', '96829', '16363', '10455', '49290', '99992', '98229', '66356', '59087', '73998', '25986', '4279', '56790', '69540', '588', '36620', '60358', '45056', '89297', '42740', '8323', '19245', '82417', '41431', '699', '11554', '73910', '44491', '56019', '68901', '45816', '68126', '89379', '23885', '13263', '56395', '73130', '19089', '23771', '10335', '48547', '16903', '6453', '33560', '89668', '38159', '43177', '90655', '49712', '62', '66920', '34180', '12150', '48564', '39538', '85026', '87195', '14928', '8956', '71157', '53287', '39161', '67583', '83309', '92054', '86977', '56188', '15229', '88170', '60894', '58497', '89254', '40082', '86890', '60161', '97291', '45878', '23368', '14577', '92870', '37017', '97356', '99426', '76061', '89186', '99751', '85153', '61580', '39360', '90107', '25603', '26798', '76224', '6469', '7912', '69838', '16404', '67497', '28965', '80836', '80365', '91249', '48713', '17113', '33090', '40793', '70450', '66689', '83698', '17802', '43869', '13355', '18959', '79411', '87930', '9265', '37504', '44876', '97234', '94149', '35040', '22049', '49248', '6535', '36080', '28346', '94437', '78319', '17961', '89056', '56161', '35810', '41632', '45494', '53351', '89729', '99510', '51584', '59688', '6193', '70809', '51093', '92589', '90247', '34910', '78235', '17362', '49423', '63324', '525', '37638', '72325', '89356', '15298', '59116', '17848', '65429', '27029', '84781', '70247', '8825', '35082', '70451', '22522', '58125', '91879', '90531', '2478', '463', '37902', '54405', '87267', '72688', '22803', '33134', '35177', '84551', '44974', '88375', '76407', '27774', '33849', '19915', '82014', '80434', '26380', '48777', '53811', '14838', '26829', '56441', '99869', '49574', '85476', '19723', '16907', '4018', '37338', '78510', '47912', '13030', '65277', '95716', '67363', '21393', '89887', '78842', '81650', '903', '17436', '30704', '49223', '27198', '25500', '52214', '54258', '70082', '53950', '49312', '43615', '99473', '94348', '53661', '96213', '96346', '62010', '38268', '32861', '75660', '10392', '89491', '68335', '29817', '88706', '24184', '36298', '43440', '21626', '26535', '44560', '46363', '12534', '99070', '95606', '33714', '73070', '8303', '29853', '23014', '99982', '4530', '14955', '45803', '50', '90750', '30394', '81276', '95563', '47314', '58520', '91299', '88944', '54402', '67405', '29253', '47079', '71734', '99728', '17652', '13307', '35556', '18962', '26780', '17771', '53712', '60055', '37628', '35830', '90739', '61151', '41309', '27652', '3051', '53167', '98417', '19382', '36833', '75085', '65374', '87732', '30352', '31776', '32765', '97565', '92199', '49050', '29503', '51024', '18834', '8515', '24069', '96216', '10777', '90680', '18974', '68884', '85305', '36007', '56707', '4212', '47352', '34426', '13185', '92939', '95782', '70577', '58080', '98279', '3906', '5065', '56896', '16382', '31273', '17117', '98602', '12786', '24086', '63970', '72756', '35798', '82367', '7356', '53398', '68503', '2962', '16425', '67334', '68461', '65439', '15620', '70906', '29649', '46461', '74602', '38012', '71714', '16825', '89480', '53386', '88532', '35104', '28556', '82120', '23155', '23347', '24797', '60061', '54962', '99427', '82248', '82447', '39968', '63727', '27431', '81511', '91168', '71425', '80740', '84127', '40717', '15503', '15419', '46594', '61263', '19212', '53175', '70724', '74445', '23034', '71818', '40246', '18886', '53066', '4880', '83701', '86107', '87862', '44751', '392', '73440', '90291', '93395', '20894', '38463', '32664', '55158', '20090', '50004', '79070', '98471', '85478', '96615', '68149', '78334', '97752', '73207', '71678', '91238', '96757', '82598', '194', '35797', '45120', '60782', '28721', '17676', '78066', '60957', '11826', '51563', '50516', '16485', '47053', '31738', '48923', '23554', '96850', '42033', '73701', '78607', '45979', '54571', '12415', '31693', '15356', '36902', '9126', '3767', '3295', '90402', '24005', '95350', '67033', '49137', '72606', '51899', '17522', '31957', '44641', '53982', '23767', '68257', '15766', '19995', '2107', '48788', '11765', '91055', '46576', '54651', '50381', '62827', '73636', '46606', '98753', '37631', '70441', '87916', '66983', '33870', '31125', '12904', '57040', '4874', '58632', '42037', '18782', '5998', '18974', '57949', '81010', '90407', '99874', '20462', '89949', '10952', '71454', '95130', '46115', '3518', '13384', '69039', '79482', '22076', '59782', '32042', '40930', '60243', '29298', '6790', '46985', '44398', '85631', '14380', '66179', '2629', '32126', '49833', '14118', '58492', '31493', '81172', '96638', '8745', '89663', '76842', '78633', '41373', '83721', '42886', '11123', '32739', '11051', '1303', '92314', '83324', '85600', '44276', '69064', '56125', '84650', '31028', '12628', '14502', '64764', '39405', '44855', '79046', '51716', '46824', '83389', '1941', '1257', '9280', '73176', '84729', '2579', '63366', '22606', '35541', '51096', '13447', '18355', '68037', '28436', '94116', '81070', '78355', '67897', '5296', '32742', '77645', '91853', '18767', '67949', '40963', '5792', '17278', '25597', '41884', '80829', '7099', '18645', '60295', '12082', '81800', '78415', '18082', '38789', '16295', '72377', '74949', '55583', '66853', '15402', '72977', '15123', '99434', '34999', '21687', '76049', '42987', '83748', '88256', '66688', '21766', '20304', '29271', '10069', '19822', '11792', '42526', '74143', '17289', '30253', '6367', '20888', '12975', '94073', '98639', '30134', '26320', '65507', '69002', '53120', '4550', '38893', '18954', '38283', '54863', '17698', '99670', '10521', '92467', '60994', '18052', '48673', '35811', '87282', '62706', '16061', '53112', '22652', '37780', '55662', '26331', '49410', '79074', '10623', '69577', '79613', '9491', '31229', '43922', '84231', '58409', '36386', '46875', '74431', '76735', '38776', '23350', '7314', '9079', '51519', '98544', '70216', '63380', '90381', '1295', '46901', '58225', '55339', '89918', '75522', '35431', '89460', '49552', '89302', '23068', '28493', '3042', '25194', '59520', '9810', '95706', '81297', '89638', '54794', '94527', '45262', '97932', '78685', '6947', '22818', '48700', '9153', '12289', '22011', '58825', '93854', '65438', '4509', '33741', '28208', '69061', '48578', '40247', '77725', '31837', '39003', '69363', '78113', '76398', '97262', '67795', <query>SELECT count() FROM numbers(1000) WHERE toString(number) IN ('41577', '83972', '51697', '50014', '37553', '93459', '87438', '95971', '83186', '74326', '67871', '50406', '83678', '29655', '18580', '83905', '61518', '29059', '56700', '82787', '98672', '30884', '81822', '39850', '80852', '57627', '91346', '64522', '17781', '49467', '41099', '41929', '85618', '91389', '68564', '91769', '81219', '52218', '37220', '97097', '2129', '9886', '52049', '34847', '25364', '36429', '76897', '71868', '58121', '71199', '84819', '69991', '34046', '64507', '34892', '24228', '36986', '28588', '51159', '53444', '80531', '9941', '20256', '48103', '32565', '62890', '5379', '60302', '46434', '3205', '18821', '31030', '19794', '71557', '71703', '15024', '14004', '82164', '95659', '40227', '83358', '24395', '9610', '19814', '48491', '66412', '16012', '71586', '42143', '51103', '24463', '89949', '35694', '39193', '63904', '40489', '77144', '94014', '84836', '9980', '46554', '43905', '25588', '25205', '72624', '10249', '35888', '98478', '99030', '26834', '31', '81499', '14847', '82997', '92357', '92893', '17426', '56630', '22252', '68119', '62710', '8740', '82144', '79916', '23391', '30192', '99271', '96435', '44237', '98327', '69481', '16691', '13643', '84554', '38571', '70926', '99283', '79000', '20926', '86495', '4834', '1222', '39486', '57697', '58002', '40790', '15623', '3999', '31515', '12694', '26143', '35951', '54085', '97534', '35329', '73535', '88715', '29572', '75799', '45166', '32066', '48023', '69523', '93150', '8740', '96790', '15534', '63252', '5142', '67045', '93992', '16663', '292', '63924', '6588', '12190', '31506', '69590', '35394', '55168', '65223', '79183', '32600', '69676', '28316', '72111', '53531', '15073', '41127', '73451', '24725', '61647', '65315', '41143', '26493', '95608', '34407', '76098', '53105', '83691', '48755', '35696', '62587', '81826', '3963', '45766', '82751', '12430', '97685', '29919', '78155', '71636', '50215', '89734', '9892', '47151', '54855', '3428', '9712', '52592', '2403', '79602', '81243', '79859', '57361', '82000', '42107', '28860', '99591', '28296', '57337', '64969', '32332', '25535', '30924', '21313', '32554', '17342', '87311', '19825', '24898', '61323', '83209', '79322', '79009', '50746', '33396', '62033', '16548', '17427', '24073', '34640', '52368', '4724', '80408', '40', '33787', '16666', '19665', '86751', '27264', '2241', '88134', '53566', '10589', '79711', '92823', '58972', '91767', '60885', '51659', '7867', '96849', '30360', '20914', '9584', '1250', '22871', '23282', '99312', '4683', '33429', '68361', '82614', '81440', '47863', '69790', '11968', '75210', '66854', '37002', '61142', '71514', '1588', '42336', '11069', '26291', '2261', '71056', '13492', '9133', '91216', '72207', '71586', '86535', '83898', '24392', '45384', '48545', '61972', '503', '80180', '35834', '97025', '70411', '55039', '35430', '27631', '82533', '96831', '74077', '42533', '14451', '26943', '53783', '69489', '71969', '8432', '37230', '61348', '19472', '59115', '9886', '50951', '57109', '7141', '1902', '84130', '4323', '55889', '47784', '2220', '75988', '66988', '63721', '8131', '95601', '95207', '2311', '26541', '50991', '6717', '2969', '71857', '51034', '65958', '94716', '90275', '21012', '46859', '7984', '31131', '46457', '69578', '44540', '7294', '80117', '9925', '60155', '90608', '82684', '32193', '87071', '28006', '87604', '24501', '79087', '2848', '29237', '11221', '81319', '40966', '87641', '35325', '78705', '88636', '78717', '62831', '56390', '99271', '43821', '14453', '17923', '62695', '77322', '21038', '67677', '41271', '4376', '65426', '46091', '19887', '97251', '55583', '58763', '3826', '35037', '73533', '64267', '82319', '9836', '42622', '96829', '16363', '10455', '49290', '99992', '98229', '66356', '59087', '73998', '25986', '4279', '56790', '69540', '588', '36620', '60358', '45056', '89297', '42740', '8323', '19245', '82417', '41431', '699', '11554', '73910', '44491', '56019', '68901', '45816', '68126', '89379', '23885', '13263', '56395', '73130', '19089', '23771', '10335', '48547', '16903', '6453', '33560', '89668', '38159', '43177', '90655', '49712', '62', '66920', '34180', '12150', '48564', '39538', '85026', '87195', '14928', '8956', '71157', '53287', '39161', '67583', '83309', '92054', '86977', '56188', '15229', '88170', '60894', '58497', '89254', '40082', '86890', '60161', '97291', '45878', '23368', '14577', '92870', '37017', '97356', '99426', '76061', '89186', '99751', '85153', '61580', '39360', '90107', '25603', '26798', '76224', '6469', '7912', '69838', '16404', '67497', '28965', '80836', '80365', '91249', '48713', '17113', '33090', '40793', '70450', '66689', '83698', '17802', '43869', '13355', '18959', '79411', '87930', '9265', '37504', '44876', '97234', '94149', '35040', '22049', '49248', '6535', '36080', '28346', '94437', '78319', '17961', '89056', '56161', '35810', '41632', '45494', '53351', '89729', '99510', '51584', '59688', '6193', '70809', '51093', '92589', '90247', '34910', '78235', '17362', '49423', '63324', '525', '37638', '72325', '89356', '15298', '59116', '17848', '65429', '27029', '84781', '70247', '8825', '35082', '70451', '22522', '58125', '91879', '90531', '2478', '463', '37902', '54405', '87267', '72688', '22803', '33134', '35177', '84551', '44974', '88375', '76407', '27774', '33849', '19915', '82014', '80434', '26380', '48777', '53811', '14838', '26829', '56441', '99869', '49574', '85476', '19723', '16907', '4018', '37338', '78510', '47912', '13030', '65277', '95716', '67363', '21393', '89887', '78842', '81650', '903', '17436', '30704', '49223', '27198', '25500', '52214', '54258', '70082', '53950', '49312', '43615', '99473', '94348', '53661', '96213', '96346', '62010', '38268', '32861', '75660', '10392', '89491', '68335', '29817', '88706', '24184', '36298', '43440', '21626', '26535', '44560', '46363', '12534', '99070', '95606', '33714', '73070', '8303', '29853', '23014', '99982', '4530', '14955', '45803', '50', '90750', '30394', '81276', '95563', '47314', '58520', '91299', '88944', '54402', '67405', '29253', '47079', '71734', '99728', '17652', '13307', '35556', '18962', '26780', '17771', '53712', '60055', '37628', '35830', '90739', '61151', '41309', '27652', '3051', '53167', '98417', '19382', '36833', '75085', '65374', '87732', '30352', '31776', '32765', '97565', '92199', '49050', '29503', '51024', '18834', '8515', '24069', '96216', '10777', '90680', '18974', '68884', '85305', '36007', '56707', '4212', '47352', '34426', '13185', '92939', '95782', '70577', '58080', '98279', '3906', '5065', '56896', '16382', '31273', '17117', '98602', '12786', '24086', '63970', '72756', '35798', '82367', '7356', '53398', '68503', '2962', '16425', '67334', '68461', '65439', '15620', '70906', '29649', '46461', '74602', '38012', '71714', '16825', '89480', '53386', '88532', '35104', '28556', '82120', '23155', '23347', '24797', '60061', '54962', '99427', '82248', '82447', '39968', '63727', '27431', '81511', '91168', '71425', '80740', '84127', '40717', '15503', '15419', '46594', '61263', '19212', '53175', '70724', '74445', '23034', '71818', '40246', '18886', '53066', '4880', '83701', '86107', '87862', '44751', '392', '73440', '90291', '93395', '20894', '38463', '32664', '55158', '20090', '50004', '79070', '98471', '85478', '96615', '68149', '78334', '97752', '73207', '71678', '91238', '96757', '82598', '194', '35797', '45120', '60782', '28721', '17676', '78066', '60957', '11826', '51563', '50516', '16485', '47053', '31738', '48923', '23554', '96850', '42033', '73701', '78607', '45979', '54571', '12415', '31693', '15356', '36902', '9126', '3767', '3295', '90402', '24005', '95350', '67033', '49137', '72606', '51899', '17522', '31957', '44641', '53982', '23767', '68257', '15766', '19995', '2107', '48788', '11765', '91055', '46576', '54651', '50381', '62827', '73636', '46606', '98753', '37631', '70441', '87916', '66983', '33870', '31125', '12904', '57040', '4874', '58632', '42037', '18782', '5998', '18974', '57949', '81010', '90407', '99874', '20462', '89949', '10952', '71454', '95130', '46115', '3518', '13384', '69039', '79482', '22076', '59782', '32042', '40930', '60243', '29298', '6790', '46985', '44398', '85631', '14380', '66179', '2629', '32126', '49833', '14118', '58492', '31493', '81172', '96638', '8745', '89663', '76842', '78633', '41373', '83721', '42886', '11123', '32739', '11051', '1303', '92314', '83324', '85600', '44276', '69064', '56125', '84650', '31028', '12628', '14502', '64764', '39405', '44855', '79046', '51716', '46824', '83389', '1941', '1257', '9280', '73176', '84729', '2579', '63366', '22606', '35541', '51096', '13447', '18355', '68037', '28436', '94116', '81070', '78355', '67897', '5296', '32742', '77645', '91853', '18767', '67949', '40963', '5792', '17278', '25597', '41884', '80829', '7099', '18645', '60295', '12082', '81800', '78415', '18082', '38789', '16295', '72377', '74949', '55583', '66853', '15402', '72977', '15123', '99434', '34999', '21687', '76049', '42987', '83748', '88256', '66688', '21766', '20304', '29271', '10069', '19822', '11792', '42526', '74143', '17289', '30253', '6367', '20888', '12975', '94073', '98639', '30134', '26320', '65507', '69002', '53120', '4550', '38893', '18954', '38283', '54863', '17698', '99670', '10521', '92467', '60994', '18052', '48673', '35811', '87282', '62706', '16061', '53112', '22652', '37780', '55662', '26331', '49410', '79074', '10623', '69577', '79613', '9491', '31229', '43922', '84231', '58409', '36386', '46875', '74431', '76735', '38776', '23350', '7314', '9079', '51519', '98544', '70216', '63380', '90381', '1295', '46901', '58225', '55339', '89918', '75522', '35431', '89460', '49552', '89302', '23068', '28493', '3042', '25194', '59520', '9810', '95706', '81297', '89638', '54794', '94527', '45262', '97932', '78685', '6947', '22818', '48700', '9153', '12289', '22011', '58825', '93854', '65438', '4509', '33741', '28208', '69061', '48578', '40247', '77725', '31837', '39003', '69363', '78113', '76398', '97262', '67795',
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册