Merge remote-tracking branch 'refs/remotes/upstream/master' into vfs_log

This commit is contained in:
Alexander Burmak 2020-01-10 15:04:20 +03:00
commit ec1a490935
70 changed files with 1997 additions and 1675 deletions

2
.github/CODEOWNERS vendored
View File

@ -1,4 +1,2 @@
dbms/* @ClickHouse/core-assigner
utils/* @ClickHouse/core-assigner
docs/* @ClickHouse/docs
docs/zh/* @ClickHouse/docs-zh

File diff suppressed because it is too large Load Diff

View File

@ -11,3 +11,7 @@ ClickHouse is an open-source column-oriented database management system that all
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
* [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-february-meetup-registration-88496227599) on February 5.

View File

@ -13,12 +13,12 @@ set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS})
# Minimal supported SDK version
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mmacosx-version-min=10.14")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mmacosx-version-min=10.14")
set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -mmacosx-version-min=10.14")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mmacosx-version-min=10.15")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mmacosx-version-min=10.15")
set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -mmacosx-version-min=10.15")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -mmacosx-version-min=10.14")
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -mmacosx-version-min=10.14")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -mmacosx-version-min=10.15")
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -mmacosx-version-min=10.15")
# Global libraries

2
contrib/googletest vendored

@ -1 +1 @@
Subproject commit d175c8bf823e709d570772b038757fadf63bc632
Subproject commit 703bd9caab50b139428cea1aaff9974ebee5742e

View File

@ -546,7 +546,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors(size_t num_threads)
auto & pipeline = state.io.pipeline;
if (pipeline.getMaxThreads())
num_threads = pipeline.getMaxThreads();
num_threads = std::min(num_threads, pipeline.getMaxThreads());
/// Send header-block, to allow client to prepare output format for data to send.
{

View File

@ -101,9 +101,6 @@ namespace
public:
void add(const ASTPtr & condition, bool is_restrictive)
{
if (!condition)
return;
if (is_restrictive)
restrictions.push_back(condition);
else
@ -139,16 +136,20 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_
for (auto index : ext::range_with_static_cast<ConditionIndex>(0, MAX_CONDITION_INDEX))
{
parsed_conditions[index] = nullptr;
const String & condition = policy->conditions[index];
if (condition.empty())
continue;
auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index);
auto previous_it = std::find(previous_range.first, previous_range.second, condition);
if (previous_it != previous_range.second)
{
/// The condition is already parsed before.
parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first];
continue;
}
else
{
/// Try to parse the condition.
try
{
@ -164,7 +165,6 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_
}
}
}
}
bool RowPolicyContextFactory::PolicyInfo::canUseWithContext(const RowPolicyContext & context) const
@ -290,6 +290,7 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context
auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}];
mixers.policy_ids.push_back(policy_id);
for (auto index : ext::range(0, MAX_CONDITION_INDEX))
if (info.parsed_conditions[index])
mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive());
}
}

View File

@ -301,10 +301,6 @@ struct Codec
: codec_statement(std::move(codec_statement_)),
expected_compression_ratio(expected_compression_ratio_)
{}
Codec()
: Codec(std::string())
{}
};
@ -314,23 +310,12 @@ struct CodecTestSequence
std::vector<char> serialized_data;
DataTypePtr data_type;
CodecTestSequence()
: name(),
serialized_data(),
data_type()
{}
CodecTestSequence(std::string name_, std::vector<char> serialized_data_, DataTypePtr data_type_)
: name(name_),
serialized_data(serialized_data_),
data_type(data_type_)
{}
CodecTestSequence(const CodecTestSequence &) = default;
CodecTestSequence & operator=(const CodecTestSequence &) = default;
CodecTestSequence(CodecTestSequence &&) = default;
CodecTestSequence & operator=(CodecTestSequence &&) = default;
CodecTestSequence & append(const CodecTestSequence & other)
{
assert(data_type->equals(*other.data_type));
@ -819,24 +804,6 @@ std::vector<CodecTestSequence> generatePyramidOfSequences(const size_t sequences
return sequences;
};
// Just as if all sequences from generatePyramidOfSequences were appended to one-by-one to the first one.
template <typename T, typename Generator>
CodecTestSequence generatePyramidSequence(const size_t sequences_count, Generator && generator, const char* generator_name)
{
CodecTestSequence sequence;
sequence.data_type = makeDataType<T>();
sequence.serialized_data.reserve(sequences_count * sequences_count * sizeof(T));
for (size_t i = 1; i < sequences_count; ++i)
{
std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i);
sequence.append(generateSeq<T>(std::forward<decltype(generator)>(generator), name.c_str(), 0, i));
}
return sequence;
};
// helper macro to produce human-friendly sequence name from generator
#define G(generator) generator, #generator
@ -853,17 +820,17 @@ const auto DefaultCodecsToTest = ::testing::Values(
// test cases
///////////////////////////////////////////////////////////////////////////////////////////////////
INSTANTIATE_TEST_CASE_P(Simple,
INSTANTIATE_TEST_SUITE_P(Simple,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
::testing::Values(
makeSeq<Float64>(1, 2, 3, 5, 7, 11, 13, 17, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61, 67, 71, 73, 79, 83, 89, 97)
)
),
)
);
INSTANTIATE_TEST_CASE_P(SmallSequences,
INSTANTIATE_TEST_SUITE_P(SmallSequences,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -877,10 +844,10 @@ INSTANTIATE_TEST_CASE_P(SmallSequences,
+ generatePyramidOfSequences<UInt32>(42, G(SequentialGenerator(1)))
+ generatePyramidOfSequences<UInt64>(42, G(SequentialGenerator(1)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(Mixed,
INSTANTIATE_TEST_SUITE_P(Mixed,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -894,10 +861,10 @@ INSTANTIATE_TEST_CASE_P(Mixed,
generateSeq<UInt32>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt32>(G(SequentialGenerator(1)), 1, 1001),
generateSeq<UInt64>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt64>(G(SequentialGenerator(1)), 1, 1001)
)
),
)
);
INSTANTIATE_TEST_CASE_P(SameValueInt,
INSTANTIATE_TEST_SUITE_P(SameValueInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -911,10 +878,10 @@ INSTANTIATE_TEST_CASE_P(SameValueInt,
generateSeq<UInt32>(G(SameValueGenerator(1000))),
generateSeq<UInt64>(G(SameValueGenerator(1000)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SameNegativeValueInt,
INSTANTIATE_TEST_SUITE_P(SameNegativeValueInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -928,10 +895,10 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueInt,
generateSeq<UInt32>(G(SameValueGenerator(-1000))),
generateSeq<UInt64>(G(SameValueGenerator(-1000)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SameValueFloat,
INSTANTIATE_TEST_SUITE_P(SameValueFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -942,10 +909,10 @@ INSTANTIATE_TEST_CASE_P(SameValueFloat,
generateSeq<Float32>(G(SameValueGenerator(M_E))),
generateSeq<Float64>(G(SameValueGenerator(M_E)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat,
INSTANTIATE_TEST_SUITE_P(SameNegativeValueFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -956,10 +923,10 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat,
generateSeq<Float32>(G(SameValueGenerator(-1 * M_E))),
generateSeq<Float64>(G(SameValueGenerator(-1 * M_E)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SequentialInt,
INSTANTIATE_TEST_SUITE_P(SequentialInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -973,12 +940,12 @@ INSTANTIATE_TEST_CASE_P(SequentialInt,
generateSeq<UInt32>(G(SequentialGenerator(1))),
generateSeq<UInt64>(G(SequentialGenerator(1)))
)
),
)
);
// -1, -2, -3, ... etc for signed
// 0xFF, 0xFE, 0xFD, ... for unsigned
INSTANTIATE_TEST_CASE_P(SequentialReverseInt,
INSTANTIATE_TEST_SUITE_P(SequentialReverseInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -992,10 +959,10 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseInt,
generateSeq<UInt32>(G(SequentialGenerator(-1))),
generateSeq<UInt64>(G(SequentialGenerator(-1)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SequentialFloat,
INSTANTIATE_TEST_SUITE_P(SequentialFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1006,10 +973,10 @@ INSTANTIATE_TEST_CASE_P(SequentialFloat,
generateSeq<Float32>(G(SequentialGenerator(M_E))),
generateSeq<Float64>(G(SequentialGenerator(M_E)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(SequentialReverseFloat,
INSTANTIATE_TEST_SUITE_P(SequentialReverseFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1020,10 +987,10 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseFloat,
generateSeq<Float32>(G(SequentialGenerator(-1 * M_E))),
generateSeq<Float64>(G(SequentialGenerator(-1 * M_E)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(MonotonicInt,
INSTANTIATE_TEST_SUITE_P(MonotonicInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -1037,10 +1004,10 @@ INSTANTIATE_TEST_CASE_P(MonotonicInt,
generateSeq<UInt32>(G(MonotonicGenerator(1, 5))),
generateSeq<UInt64>(G(MonotonicGenerator(1, 5)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(MonotonicReverseInt,
INSTANTIATE_TEST_SUITE_P(MonotonicReverseInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -1054,10 +1021,10 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseInt,
generateSeq<UInt32>(G(MonotonicGenerator(-1, 5))),
generateSeq<UInt64>(G(MonotonicGenerator(-1, 5)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(MonotonicFloat,
INSTANTIATE_TEST_SUITE_P(MonotonicFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1067,10 +1034,10 @@ INSTANTIATE_TEST_CASE_P(MonotonicFloat,
generateSeq<Float32>(G(MonotonicGenerator<Float32>(M_E, 5))),
generateSeq<Float64>(G(MonotonicGenerator<Float64>(M_E, 5)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat,
INSTANTIATE_TEST_SUITE_P(MonotonicReverseFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1080,10 +1047,10 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat,
generateSeq<Float32>(G(MonotonicGenerator<Float32>(-1 * M_E, 5))),
generateSeq<Float64>(G(MonotonicGenerator<Float64>(-1 * M_E, 5)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(RandomInt,
INSTANTIATE_TEST_SUITE_P(RandomInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -1093,10 +1060,10 @@ INSTANTIATE_TEST_CASE_P(RandomInt,
generateSeq<UInt32>(G(RandomGenerator<UInt32>(0, 0, 1000'000'000))),
generateSeq<UInt64>(G(RandomGenerator<UInt64>(0, 0, 1000'000'000)))
)
),
)
);
INSTANTIATE_TEST_CASE_P(RandomishInt,
INSTANTIATE_TEST_SUITE_P(RandomishInt,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -1108,10 +1075,10 @@ INSTANTIATE_TEST_CASE_P(RandomishInt,
generateSeq<Float32>(G(RandomishGenerator)),
generateSeq<Float64>(G(RandomishGenerator))
)
),
)
);
INSTANTIATE_TEST_CASE_P(RandomishFloat,
INSTANTIATE_TEST_SUITE_P(RandomishFloat,
CodecTest,
::testing::Combine(
DefaultCodecsToTest,
@ -1119,11 +1086,11 @@ INSTANTIATE_TEST_CASE_P(RandomishFloat,
generateSeq<Float32>(G(RandomishGenerator)),
generateSeq<Float64>(G(RandomishGenerator))
)
),
)
);
// Double delta overflow case, deltas are out of bounds for target type
INSTANTIATE_TEST_CASE_P(OverflowInt,
INSTANTIATE_TEST_SUITE_P(OverflowInt,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1136,10 +1103,10 @@ INSTANTIATE_TEST_CASE_P(OverflowInt,
generateSeq<UInt64>(G(MinMaxGenerator())),
generateSeq<Int64>(G(MinMaxGenerator()))
)
),
)
);
INSTANTIATE_TEST_CASE_P(OverflowFloat,
INSTANTIATE_TEST_SUITE_P(OverflowFloat,
CodecTest,
::testing::Combine(
::testing::Values(
@ -1152,7 +1119,7 @@ INSTANTIATE_TEST_CASE_P(OverflowFloat,
generateSeq<Float32>(G(FFand0Generator())),
generateSeq<Float64>(G(FFand0Generator()))
)
),
)
);
template <typename ValueType>
@ -1189,7 +1156,7 @@ auto DDCompatibilityTestSequence()
#define BIN_STR(x) std::string{x, sizeof(x) - 1}
INSTANTIATE_TEST_CASE_P(DoubleDelta,
INSTANTIATE_TEST_SUITE_P(DoubleDelta,
CodecTest_Compatibility,
::testing::Combine(
::testing::Values(Codec("DoubleDelta")),
@ -1227,7 +1194,7 @@ INSTANTIATE_TEST_CASE_P(DoubleDelta,
BIN_STR("\x94\xd4\x00\x00\x00\x98\x01\x00\x00\x08\x00\x33\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00\xfc\x00\x00\x00\x04\x00\x06\xbe\x4f\xbf\xff\xd6\x0c\xff\x00\x00\x00\x01\x00\x00\x00\x03\xf8\x00\x00\x00\x08\x00\x00\x00\x0f\xc0\x00\x00\x00\x3f\xff\xff\xff\xfb\xff\xff\xff\xfb\xe0\x00\x00\x01\xc0\x00\x00\x06\x9f\x80\x00\x00\x0a\x00\x00\x00\x34\xf3\xff\xff\xff\xe7\x9f\xff\xff\xff\x7e\x00\x00\x00\x00\xff\xff\xff\xfd\xf0\x00\x00\x00\x07\xff\xff\xff\xf0")
},
})
),
)
);
template <typename ValueType>
@ -1263,7 +1230,7 @@ auto GCompatibilityTestSequence()
return generateSeq<ValueType>(G(PrimesWithMultiplierGenerator(intExp10(sizeof(ValueType)))), 0, 42);
}
INSTANTIATE_TEST_CASE_P(Gorilla,
INSTANTIATE_TEST_SUITE_P(Gorilla,
CodecTest_Compatibility,
::testing::Combine(
::testing::Values(Codec("Gorilla")),
@ -1301,14 +1268,31 @@ INSTANTIATE_TEST_CASE_P(Gorilla,
BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10")
},
})
),
)
);
// These 'tests' try to measure performance of encoding and decoding and hence only make sence to be run locally,
// also they require pretty big data to run agains and generating this data slows down startup of unit test process.
// So un-comment only at your discretion.
//INSTANTIATE_TEST_CASE_P(DoubleDelta,
// Just as if all sequences from generatePyramidOfSequences were appended to one-by-one to the first one.
//template <typename T, typename Generator>
//CodecTestSequence generatePyramidSequence(const size_t sequences_count, Generator && generator, const char* generator_name)
//{
// CodecTestSequence sequence;
// sequence.data_type = makeDataType<T>();
// sequence.serialized_data.reserve(sequences_count * sequences_count * sizeof(T));
//
// for (size_t i = 1; i < sequences_count; ++i)
// {
// std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i);
// sequence.append(generateSeq<T>(std::forward<decltype(generator)>(generator), name.c_str(), 0, i));
// }
//
// return sequence;
//};
//INSTANTIATE_TEST_SUITE_P(DoubleDelta,
// CodecTest_Performance,
// ::testing::Combine(
// ::testing::Values(Codec("DoubleDelta")),
@ -1325,7 +1309,7 @@ INSTANTIATE_TEST_CASE_P(Gorilla,
// ),
//);
//INSTANTIATE_TEST_CASE_P(Gorilla,
//INSTANTIATE_TEST_SUITE_P(Gorilla,
// CodecTest_Performance,
// ::testing::Combine(
// ::testing::Values(Codec("Gorilla")),

View File

@ -121,19 +121,19 @@ struct SortCursorHelper
SortCursorImpl * operator-> () { return impl; }
const SortCursorImpl * operator-> () const { return impl; }
bool greater(const SortCursorHelper & rhs) const
bool ALWAYS_INLINE greater(const SortCursorHelper & rhs) const
{
return derived().greaterAt(rhs.derived(), impl->pos, rhs.impl->pos);
}
/// Inverted so that the priority queue elements are removed in ascending order.
bool operator< (const SortCursorHelper & rhs) const
bool ALWAYS_INLINE operator< (const SortCursorHelper & rhs) const
{
return derived().greater(rhs.derived());
}
/// Checks that all rows in the current block of this cursor are less than or equal to all the rows of the current block of another cursor.
bool totallyLessOrEquals(const SortCursorHelper & rhs) const
bool ALWAYS_INLINE totallyLessOrEquals(const SortCursorHelper & rhs) const
{
if (impl->rows == 0 || rhs.impl->rows == 0)
return false;
@ -149,7 +149,7 @@ struct SortCursor : SortCursorHelper<SortCursor>
using SortCursorHelper<SortCursor>::SortCursorHelper;
/// The specified row of this cursor is greater than the specified row of another cursor.
bool greaterAt(const SortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const
bool ALWAYS_INLINE greaterAt(const SortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const
{
for (size_t i = 0; i < impl->sort_columns_size; ++i)
{
@ -172,7 +172,7 @@ struct SimpleSortCursor : SortCursorHelper<SimpleSortCursor>
{
using SortCursorHelper<SimpleSortCursor>::SortCursorHelper;
bool greaterAt(const SimpleSortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const
bool ALWAYS_INLINE greaterAt(const SimpleSortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const
{
const auto & desc = impl->desc[0];
int direction = desc.direction;
@ -188,7 +188,7 @@ struct SortCursorWithCollation : SortCursorHelper<SortCursorWithCollation>
{
using SortCursorHelper<SortCursorWithCollation>::SortCursorHelper;
bool greaterAt(const SortCursorWithCollation & rhs, size_t lhs_pos, size_t rhs_pos) const
bool ALWAYS_INLINE greaterAt(const SortCursorWithCollation & rhs, size_t lhs_pos, size_t rhs_pos) const
{
for (size_t i = 0; i < impl->sort_columns_size; ++i)
{
@ -243,7 +243,7 @@ public:
Cursor & nextChild() { return queue[nextChildIndex()]; }
void next()
void ALWAYS_INLINE next()
{
assert(isValid());
@ -283,7 +283,7 @@ private:
/// Cache comparison between first and second child if the order in queue has not been changed.
size_t next_idx = 0;
size_t nextChildIndex()
size_t ALWAYS_INLINE nextChildIndex()
{
if (next_idx == 0)
{
@ -300,7 +300,7 @@ private:
/// Why cannot simply use std::priority_queue?
/// - because it doesn't support updating the top element and requires pop and push instead.
/// Also look at "Boost.Heap" library.
void updateTop()
void ALWAYS_INLINE updateTop()
{
size_t size = queue.size();
if (size < 2)

View File

@ -120,7 +120,7 @@ TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal128)
}
// Intentionally small values that fit into 32-bit in order to cover Decimal32, Decimal64 and Decimal128 with single set of data.
INSTANTIATE_TEST_CASE_P(Basic,
INSTANTIATE_TEST_SUITE_P(Basic,
DecimalUtilsSplitAndCombineTest,
::testing::ValuesIn(std::initializer_list<DecimalUtilsSplitAndCombineTestParam>{
{
@ -168,5 +168,5 @@ INSTANTIATE_TEST_CASE_P(Basic,
89
}
}
}
),);
})
);

View File

@ -104,7 +104,7 @@ TEST_P(MostSubtypeTest, getLeastSupertype)
}
}
INSTANTIATE_TEST_CASE_P(data_type,
INSTANTIATE_TEST_SUITE_P(data_type,
LeastSuperTypeTest,
::testing::ValuesIn(
std::initializer_list<TypesTestCase>{
@ -159,10 +159,10 @@ INSTANTIATE_TEST_CASE_P(data_type,
{"Tuple(Int64,Int8) Tuple(UInt64)", nullptr},
{"Array(Int64) Array(String)", nullptr},
}
),
)
);
INSTANTIATE_TEST_CASE_P(data_type,
INSTANTIATE_TEST_SUITE_P(data_type,
MostSubtypeTest,
::testing::ValuesIn(
std::initializer_list<TypesTestCase>{
@ -210,5 +210,6 @@ INSTANTIATE_TEST_CASE_P(data_type,
{"Int8 String", nullptr},
{"Nothing", nullptr},
{"FixedString(16) FixedString(8) String", nullptr},
}),
}
)
);

View File

@ -238,7 +238,7 @@ struct StringSource
size_t getElementSize() const
{
return offsets[row_num] - prev_offset;
return offsets[row_num] - prev_offset - 1;
}
Slice getWhole() const

View File

@ -40,6 +40,8 @@ public:
bool isVariadic() const override { return true; }
bool isStateful() const override { return true; }
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return false; }

View File

@ -29,7 +29,7 @@ const std::pair<LogsLevel, Message::Priority> & convertLogLevel(Aws::Utils::Logg
return mapping.at(log_level);
}
class AWSLogger : public Aws::Utils::Logging::LogSystemInterface
class AWSLogger final : public Aws::Utils::Logging::LogSystemInterface
{
public:
~AWSLogger() final = default;

View File

@ -79,7 +79,7 @@ TEST_P(DateTime64StringParseBestEffortTest, parse)
// YYYY-MM-DD HH:MM:SS.NNNNNNNNN
INSTANTIATE_TEST_CASE_P(Basic,
INSTANTIATE_TEST_SUITE_P(Basic,
DateTime64StringParseTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
@ -130,10 +130,10 @@ INSTANTIATE_TEST_CASE_P(Basic,
1568650817'1ULL,
1
}
}),
})
);
INSTANTIATE_TEST_CASE_P(BestEffort,
INSTANTIATE_TEST_SUITE_P(BestEffort,
DateTime64StringParseBestEffortTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
@ -142,13 +142,13 @@ INSTANTIATE_TEST_CASE_P(BestEffort,
1568650817'123456ULL,
6
}
}),
})
);
// TODO: add negative test cases for invalid strings, verifying that error is reported properly
INSTANTIATE_TEST_CASE_P(Basic,
INSTANTIATE_TEST_SUITE_P(Basic,
DateTime64StringWriteTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
@ -181,6 +181,6 @@ INSTANTIATE_TEST_CASE_P(Basic,
1568650817'001ULL,
3
}
}),
})
);

View File

@ -177,7 +177,7 @@ TEST_P(BitIO, WriteAndRead)
}
}
INSTANTIATE_TEST_CASE_P(Simple,
INSTANTIATE_TEST_SUITE_P(Simple,
BitIO,
::testing::ValuesIn(std::initializer_list<TestCaseParameter>{
{
@ -221,7 +221,7 @@ INSTANTIATE_TEST_CASE_P(Simple,
"10101001 10111010 11101111 10101111 10111010 11101011 10101001 00000000 " // 256
"10101111 10111010 11101011 10101001 00001111 11110000 00001110 11111111 " // 320
}
}),
})
);
TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern)
@ -241,12 +241,13 @@ TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern)
return TestCaseParameter(test_data);
}
INSTANTIATE_TEST_CASE_P(Primes,
INSTANTIATE_TEST_SUITE_P(Primes,
BitIO,
::testing::Values(
primes_case(11, 0xFFFFFFFFFFFFFFFFULL),
primes_case(11, BIT_PATTERN)
),);
)
);
TEST(BitHelpers, maskLowBits)
{

View File

@ -72,4 +72,6 @@ private:
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database);
std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number);
using TablesWithColumnNames = std::vector<TableWithColumnNames>;
}

View File

@ -26,7 +26,6 @@
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/Set.h>
#include <Interpreters/AnalyzedJoin.h>

View File

@ -0,0 +1,79 @@
#include <Interpreters/ExtractExpressionInfoVisitor.h>
#include <Functions/FunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTSubquery.h>
namespace DB
{
void ExpressionInfoMatcher::visit(const ASTPtr & ast, Data & data)
{
if (const auto * function = ast->as<ASTFunction>())
visit(*function, ast, data);
else if (const auto * identifier = ast->as<ASTIdentifier>())
visit(*identifier, ast, data);
}
void ExpressionInfoMatcher::visit(const ASTFunction & ast_function, const ASTPtr &, Data & data)
{
if (ast_function.name == "arrayJoin")
data.is_array_join = true;
else if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
data.is_aggregate_function = true;
else
{
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context);
/// Skip lambda, tuple and other special functions
if (function && function->isStateful())
data.is_stateful_function = true;
}
}
void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
{
if (!identifier.compound())
{
for (size_t index = 0; index < data.tables.size(); ++index)
{
const auto & columns = data.tables[index].columns;
// TODO: make sure no collision ever happens
if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end())
{
data.unique_reference_tables_pos.emplace(index);
break;
}
}
}
else
{
size_t best_table_pos = 0;
if (IdentifierSemantic::chooseTable(identifier, data.tables, best_table_pos))
data.unique_reference_tables_pos.emplace(best_table_pos);
}
}
bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
{
return !node->as<ASTSubquery>();
}
bool hasStatefulFunction(const ASTPtr & node, const Context & context)
{
for (const auto & select_expression : node->children)
{
ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}};
ExpressionInfoVisitor(expression_info).visit(select_expression);
if (expression_info.is_stateful_function)
return true;
}
return false;
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
namespace DB
{
class Context;
struct ExpressionInfoMatcher
{
struct Data
{
const Context & context;
const std::vector<TableWithColumnNames> & tables;
bool is_array_join = false;
bool is_stateful_function = false;
bool is_aggregate_function = false;
std::unordered_set<size_t> unique_reference_tables_pos = {};
};
static void visit(const ASTPtr & ast, Data & data);
static bool needChildVisit(const ASTPtr & node, const ASTPtr &);
static void visit(const ASTFunction & ast_function, const ASTPtr &, Data & data);
static void visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data);
};
using ExpressionInfoVisitor = ConstInDepthNodeVisitor<ExpressionInfoMatcher, true>;
bool hasStatefulFunction(const ASTPtr & node, const Context & context);
}

View File

@ -1,16 +0,0 @@
#include <Interpreters/ExtractFunctionDataVisitor.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
void ExtractFunctionData::visit(ASTFunction & function, ASTPtr &)
{
if (AggregateFunctionFactory::instance().isAggregateFunctionName(function.name))
aggregate_functions.emplace_back(&function);
else
functions.emplace_back(&function);
}
}

View File

@ -1,25 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
struct ExtractFunctionData
{
using TypeToVisit = ASTFunction;
std::vector<ASTFunction *> functions;
std::vector<ASTFunction *> aggregate_functions;
void visit(ASTFunction & identifier, ASTPtr &);
};
using ExtractFunctionMatcher = OneTypeMatcher<ExtractFunctionData>;
using ExtractFunctionVisitor = InDepthNodeVisitor<ExtractFunctionMatcher, true>;
}

View File

@ -1,40 +0,0 @@
#include <Interpreters/FindIdentifierBestTableVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
namespace DB
{
FindIdentifierBestTableData::FindIdentifierBestTableData(const std::vector<TableWithColumnNames> & tables_)
: tables(tables_)
{
}
void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
{
const DatabaseAndTableWithAlias * best_table = nullptr;
if (!identifier.compound())
{
for (const auto & table_names : tables)
{
auto & columns = table_names.columns;
if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end())
{
// TODO: make sure no collision ever happens
if (!best_table)
best_table = &table_names.table;
}
}
}
else
{
size_t best_table_pos = 0;
if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos))
best_table = &tables[best_table_pos].table;
}
identifier_table.emplace_back(&identifier, best_table);
}
}

View File

@ -1,27 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
namespace DB
{
struct FindIdentifierBestTableData
{
using TypeToVisit = ASTIdentifier;
using IdentifierWithTable = std::pair<ASTIdentifier *, const DatabaseAndTableWithAlias *>;
const std::vector<TableWithColumnNames> & tables;
std::vector<IdentifierWithTable> identifier_table;
FindIdentifierBestTableData(const std::vector<TableWithColumnNames> & tables_);
void visit(ASTIdentifier & identifier, ASTPtr &);
};
using FindIdentifierBestTableMatcher = OneTypeMatcher<FindIdentifierBestTableData>;
using FindIdentifierBestTableVisitor = InDepthNodeVisitor<FindIdentifierBestTableMatcher, true>;
}

View File

@ -59,7 +59,13 @@ public:
using Data = Data_;
using TypeToVisit = typename Data::TypeToVisit;
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return visit_children; }
static bool needChildVisit(const ASTPtr & node, const ASTPtr &)
{
if (node && node->as<TypeToVisit>())
return visit_children;
return true;
}
static void visit(T & ast, Data & data)
{

View File

@ -488,8 +488,8 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeli
QueryPipeline InterpreterSelectQuery::executeWithProcessors()
{
QueryPipeline query_pipeline;
query_pipeline.setMaxThreads(context->getSettingsRef().max_threads);
executeImpl(query_pipeline, input, query_pipeline);
query_pipeline.setMaxThreads(max_streams);
query_pipeline.addInterpreterContext(context);
query_pipeline.addStorageHolder(storage);
return query_pipeline;
@ -503,13 +503,15 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
/// Do all AST changes here, because actions from analysis_result will be used later in readImpl.
/// PREWHERE optimization.
/// Turn off, if the table filter (row-level security) is applied.
if (storage && !context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER))
if (storage)
{
query_analyzer->makeSetsForIndex(query.where());
query_analyzer->makeSetsForIndex(query.prewhere());
/// PREWHERE optimization.
/// Turn off, if the table filter (row-level security) is applied.
if (!context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER))
{
auto optimize_prewhere = [&](auto & merge_tree)
{
SelectQueryInfo current_info;
@ -526,6 +528,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
if (const auto * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get()))
optimize_prewhere(*merge_tree_data);
}
}
if (storage && !options.only_analyze)
from_stage = storage->getQueryProcessingStage(*context);
@ -1180,7 +1183,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
if (expressions.second_stage)
{
bool need_second_distinct_pass = false;
bool need_merge_streams = false;
if (expressions.need_aggregate)
{
@ -1241,13 +1243,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
executePreLimit(pipeline);
}
if (need_second_distinct_pass
|| query.limitLength()
|| query.limitBy()
|| pipeline.hasDelayedStream())
{
bool need_merge_streams = need_second_distinct_pass || query.limitLength() || query.limitBy();
if constexpr (!pipeline_with_processors)
if (pipeline.hasDelayedStream())
need_merge_streams = true;
}
if (need_merge_streams)
{
@ -1793,6 +1793,9 @@ void InterpreterSelectQuery::executeFetchColumns(
// pipes[i].pinSources(i);
// }
for (auto & pipe : pipes)
pipe.enableQuota();
pipeline.init(std::move(pipes));
}
else
@ -1930,7 +1933,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
* 1. Parallel aggregation is done, and the results should be merged in parallel.
* 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way.
*/
bool allow_to_use_two_level_group_by = pipeline.getNumMainStreams() > 1 || settings.max_bytes_before_external_group_by != 0;
bool allow_to_use_two_level_group_by = pipeline.getNumStreams() > 1 || settings.max_bytes_before_external_group_by != 0;
Aggregator::Params params(header_before_aggregation, keys, aggregates,
overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
@ -1944,12 +1947,12 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
pipeline.dropTotalsIfHas();
/// If there are several sources, then we perform parallel aggregation
if (pipeline.getNumMainStreams() > 1)
if (pipeline.getNumStreams() > 1)
{
/// Add resize transform to uniformly distribute data between aggregating streams.
pipeline.resize(pipeline.getNumMainStreams(), true);
pipeline.resize(pipeline.getNumStreams(), true);
auto many_data = std::make_shared<ManyAggregatedData>(pipeline.getNumMainStreams());
auto many_data = std::make_shared<ManyAggregatedData>(pipeline.getNumStreams());
auto merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
@ -1971,6 +1974,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
return std::make_shared<AggregatingTransform>(header, transform_params);
});
}
pipeline.enableQuotaForCurrentStreams();
}
@ -2084,6 +2089,8 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo
pipeline.addPipe(std::move(pipe));
}
pipeline.enableQuotaForCurrentStreams();
}
@ -2317,6 +2324,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
pipeline.addPipe({ std::move(transform) });
}
pipeline.enableQuotaForCurrentStreams();
if (need_finish_sorting)
{
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
@ -2342,9 +2351,6 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
return std::make_shared<PartialSortingTransform>(header, output_order_descr, limit, do_count_rows);
});
/// If there are several streams, we merge them into one
pipeline.resize(1);
/// Merge the sorted blocks.
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
@ -2353,9 +2359,12 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
return std::make_shared<MergeSortingTransform>(
header, output_order_descr, settings.max_block_size, limit,
settings.max_bytes_before_remerge_sort,
settings.max_bytes_before_remerge_sort / pipeline.getNumStreams(),
settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data);
});
/// If there are several streams, we merge them into one
executeMergeSorted(pipeline, output_order_descr, limit);
}
@ -2417,6 +2426,8 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const
settings.max_block_size, limit);
pipeline.addPipe({ std::move(transform) });
pipeline.enableQuotaForCurrentStreams();
}
}
@ -2794,11 +2805,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets)
{
if (query_info.input_sorting_info)
{
if (pipeline.hasDelayedStream())
throw Exception("Using read in order optimization, but has delayed stream in pipeline", ErrorCodes::LOGICAL_ERROR);
executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0);
}
const Settings & settings = context->getSettingsRef();
@ -2815,7 +2822,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header)
{
/// Unify streams in case they have different headers.
/// TODO: remove previos addition of _dummy column.
/// TODO: remove previous addition of _dummy column.
if (header.columns() > 1 && header.has("_dummy"))
header.erase("_dummy");

View File

@ -1,32 +1,13 @@
#include <iostream>
#include <Common/typeid_cast.h>
#include <Storages/IStorage.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/IdentifierSemantic.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/queryToString.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/FindIdentifierBestTableVisitor.h>
#include <Interpreters/ExtractFunctionDataVisitor.h>
#include <Interpreters/getTableExpressions.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/PredicateRewriteVisitor.h>
#include <Interpreters/ExtractExpressionInfoVisitor.h>
namespace DB
@ -38,155 +19,105 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_AST;
}
namespace
{
constexpr auto and_function_name = "and";
String qualifiedName(ASTIdentifier * identifier, const String & prefix)
{
if (identifier->isShort())
return prefix + identifier->getAliasOrColumnName();
return identifier->getAliasOrColumnName();
}
}
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_)
: ast_select(ast_select_), settings(settings_), context(context_)
const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_)
: context(context_), tables_with_columns(tables_with_columns_), settings(settings_)
{
}
bool PredicateExpressionsOptimizer::optimize()
bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query)
{
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables() || ast_select->tables()->children.empty())
if (!settings.enable_optimize_predicate_expression)
return false;
if (!ast_select->where() && !ast_select->prewhere())
if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals))
tryMovePredicatesFromHavingToWhere(select_query);
if (!select_query.tables() || select_query.tables()->children.empty())
return false;
if (ast_select->array_join_expression_list())
if ((!select_query.where() && !select_query.prewhere()) || select_query.array_join_expression_list())
return false;
SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns();
const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere());
bool is_rewrite_subqueries = false;
if (!all_subquery_projection_columns.empty())
{
is_rewrite_subqueries |= optimizeImpl(ast_select->where(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE);
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE);
if (!tables_predicates.empty())
return tryRewritePredicatesToTables(select_query.refTables()->children, tables_predicates);
return false;
}
return is_rewrite_subqueries;
}
bool PredicateExpressionsOptimizer::optimizeImpl(
const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
static ASTs splitConjunctionPredicate(const std::initializer_list<const ASTPtr> & predicates)
{
/// split predicate with `and`
std::vector<ASTPtr> outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
std::vector<ASTPtr> res;
std::vector<const ASTTableExpression *> table_expressions = getTableExpressions(*ast_select);
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context);
bool is_rewrite_subquery = false;
for (auto & outer_predicate : outer_predicate_expressions)
auto remove_expression_at_index = [&res] (const size_t index)
{
if (isArrayJoinFunction(outer_predicate))
if (index < res.size() - 1)
std::swap(res[index], res.back());
res.pop_back();
};
for (const auto & predicate : predicates)
{
if (!predicate)
continue;
auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, tables_with_columns);
res.emplace_back(predicate);
/// TODO: remove origin expression
for (const auto & [subquery, projection_columns] : subqueries_projection_columns)
for (size_t idx = 0; idx < res.size();)
{
OptimizeKind optimize_kind = OptimizeKind::NONE;
if (allowPushDown(subquery, outer_predicate, projection_columns, outer_predicate_dependencies, optimize_kind))
const auto & expression = res.at(idx);
if (const auto * function = expression->as<ASTFunction>(); function && function->name == "and")
{
if (optimize_kind == OptimizeKind::NONE)
optimize_kind = expression_kind;
for (auto & child : function->arguments->children)
res.emplace_back(child);
ASTPtr inner_predicate = outer_predicate->clone();
cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate
std::vector<IdentifierWithQualifier> inner_predicate_dependencies =
getDependenciesAndQualifiers(inner_predicate, tables_with_columns);
setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies);
switch (optimize_kind)
{
case OptimizeKind::NONE: continue;
case OptimizeKind::PUSH_TO_WHERE:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::WHERE);
continue;
case OptimizeKind::PUSH_TO_HAVING:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::HAVING);
continue;
case OptimizeKind::PUSH_TO_PREWHERE:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::PREWHERE);
remove_expression_at_index(idx);
continue;
}
++idx;
}
}
}
return is_rewrite_subquery;
}
bool PredicateExpressionsOptimizer::allowPushDown(
const ASTSelectQuery * subquery,
const ASTPtr &,
const std::vector<ProjectionWithAlias> & projection_columns,
const std::vector<IdentifierWithQualifier> & dependencies,
OptimizeKind & optimize_kind)
return res;
}
std::vector<ASTs> PredicateExpressionsOptimizer::extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere)
{
if (!subquery
|| (!settings.enable_optimize_predicate_expression_to_final_subquery && subquery->final())
|| subquery->limitBy() || subquery->limitLength()
|| subquery->with() || subquery->withFill())
return false;
else
{
ASTPtr expr_list = ast_select->select();
ExtractFunctionVisitor::Data extract_data;
ExtractFunctionVisitor(extract_data).visit(expr_list);
std::vector<ASTs> tables_predicates(tables_with_columns.size());
for (const auto & subquery_function : extract_data.functions)
for (const auto & predicate_expression : splitConjunctionPredicate({where, prewhere}))
{
const auto & function = FunctionFactory::instance().tryGet(subquery_function->name, context);
ExpressionInfoVisitor::Data expression_info{.context = context, .tables = tables_with_columns};
ExpressionInfoVisitor(expression_info).visit(predicate_expression);
/// Skip lambda, tuple and other special functions
if (function && function->isStateful())
return false;
if (expression_info.is_stateful_function)
return {}; /// give up the optimization when the predicate contains stateful function
if (!expression_info.is_array_join)
{
if (expression_info.unique_reference_tables_pos.size() == 1)
tables_predicates[*expression_info.unique_reference_tables_pos.begin()].emplace_back(predicate_expression);
else if (expression_info.unique_reference_tables_pos.size() == 0)
{
for (size_t index = 0; index < tables_predicates.size(); ++index)
tables_predicates[index].emplace_back(predicate_expression);
}
}
}
const auto * ast_join = ast_select->join();
const ASTTableExpression * left_table_expr = nullptr;
const ASTTableExpression * right_table_expr = nullptr;
const ASTSelectQuery * left_subquery = nullptr;
const ASTSelectQuery * right_subquery = nullptr;
return tables_predicates; /// everything is OK, it can be optimized
}
if (ast_join)
bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_element, const std::vector<ASTs> & tables_predicates)
{
left_table_expr = ast_select
->tables()->as<ASTTablesInSelectQuery>()
->children[0]->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>();
right_table_expr = ast_select
->tables()->as<ASTTablesInSelectQuery>()
->children[1]->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>();
bool is_rewrite_tables = false;
if (left_table_expr && left_table_expr->subquery)
left_subquery = left_table_expr->subquery
->children[0]->as<ASTSelectWithUnionQuery>()
->list_of_selects->children[0]->as<ASTSelectQuery>();
if (right_table_expr && right_table_expr->subquery)
right_subquery = right_table_expr->subquery
->children[0]->as<ASTSelectWithUnionQuery>()
->list_of_selects->children[0]->as<ASTSelectQuery>();
for (size_t index = tables_element.size(); index > 0; --index)
{
size_t table_pos = index - 1;
/// NOTE: the syntactic way of pushdown has limitations and should be partially disabled in case of JOINs.
/// Let's take a look at the query:
@ -201,326 +132,84 @@ bool PredicateExpressionsOptimizer::allowPushDown(
/// It happens because the not-matching columns are replaced with a global default values on JOIN.
/// Same is true for RIGHT JOIN and FULL JOIN.
/// Check right side for LEFT'o'FULL JOIN
if (isLeftOrFull(ast_join->table_join->as<ASTTableJoin>()->kind) && right_subquery == subquery)
return false;
if (const auto & table_element = tables_element[table_pos]->as<ASTTablesInSelectQueryElement>())
{
if (table_element->table_join && isLeft(table_element->table_join->as<ASTTableJoin>()->kind))
continue; /// Skip right table optimization
if (table_element->table_join && isFull(table_element->table_join->as<ASTTableJoin>()->kind))
break; /// Skip left and right table optimization
is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos],
tables_with_columns[table_pos].columns);
if (table_element->table_join && isRight(table_element->table_join->as<ASTTableJoin>()->kind))
break; /// Skip left table optimization
}
}
return is_rewrite_tables;
}
bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const
{
if (!table_predicates.empty())
{
auto optimize_final = settings.enable_optimize_predicate_expression_to_final_subquery;
PredicateRewriteVisitor::Data data(context, table_predicates, table_column, optimize_final);
PredicateRewriteVisitor(data).visit(table_element);
return data.is_rewrite;
}
/// Check left side for RIGHT'o'FULL JOIN
if (isRightOrFull(ast_join->table_join->as<ASTTableJoin>()->kind) && left_subquery == subquery)
return false;
}
return checkDependencies(projection_columns, dependencies, optimize_kind);
}
bool PredicateExpressionsOptimizer::checkDependencies(
const std::vector<ProjectionWithAlias> & projection_columns,
const std::vector<IdentifierWithQualifier> & dependencies,
OptimizeKind & optimize_kind)
bool PredicateExpressionsOptimizer::tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query)
{
for (const auto & [identifier, prefix] : dependencies)
ASTs where_predicates;
ASTs having_predicates;
const auto & reduce_predicates = [&](const ASTs & predicates)
{
bool is_found = false;
String qualified_name = qualifiedName(identifier, prefix);
ASTPtr res = predicates[0];
for (size_t index = 1; index < predicates.size(); ++index)
res = makeASTFunction("and", res, predicates[index]);
for (const auto & [ast, alias] : projection_columns)
{
if (alias == qualified_name)
{
is_found = true;
ASTPtr projection_column = ast;
ExtractFunctionVisitor::Data extract_data;
ExtractFunctionVisitor(extract_data).visit(projection_column);
if (!extract_data.aggregate_functions.empty())
optimize_kind = OptimizeKind::PUSH_TO_HAVING;
}
}
if (!is_found)
return false;
}
return true;
}
std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(const ASTPtr & predicate_expression)
{
std::vector<ASTPtr> predicate_expressions;
if (predicate_expression)
{
predicate_expressions.emplace_back(predicate_expression);
auto remove_expression_at_index = [&predicate_expressions] (const size_t index)
{
if (index < predicate_expressions.size() - 1)
std::swap(predicate_expressions[index], predicate_expressions.back());
predicate_expressions.pop_back();
return res;
};
for (size_t idx = 0; idx < predicate_expressions.size();)
for (const auto & moving_predicate: splitConjunctionPredicate({select_query.having()}))
{
const auto expression = predicate_expressions.at(idx);
if (const auto * function = expression->as<ASTFunction>())
{
if (function->name == and_function_name)
{
for (auto & child : function->arguments->children)
predicate_expressions.emplace_back(child);
remove_expression_at_index(idx);
continue;
}
}
++idx;
}
}
return predicate_expressions;
}
std::vector<PredicateExpressionsOptimizer::IdentifierWithQualifier>
PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector<TableWithColumnNames> & tables)
{
FindIdentifierBestTableVisitor::Data find_data(tables);
FindIdentifierBestTableVisitor(find_data).visit(expression);
std::vector<IdentifierWithQualifier> dependencies;
for (const auto & [identifier, table] : find_data.identifier_table)
{
String table_alias;
if (table)
table_alias = table->getQualifiedNamePrefix();
dependencies.emplace_back(identifier, table_alias);
}
return dependencies;
}
void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate(
const std::vector<ProjectionWithAlias> & projection_columns,
const std::vector<IdentifierWithQualifier> & dependencies)
{
for (auto & [identifier, prefix] : dependencies)
{
String qualified_name = qualifiedName(identifier, prefix);
for (auto & [ast, alias] : projection_columns)
{
if (alias == qualified_name)
{
String name;
if (auto * id = ast->as<ASTIdentifier>())
{
name = id->tryGetAlias();
if (name.empty())
name = id->shortName();
}
else
{
if (ast->tryGetAlias().empty())
ast->setAlias(ast->getColumnName());
name = ast->getAliasOrColumnName();
}
identifier->setShortName(name);
}
}
}
}
bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
{
if (const auto * function = node->as<ASTFunction>())
{
if (function->name == "arrayJoin")
return true;
}
for (auto & child : node->children)
if (isArrayJoinFunction(child))
return true;
ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}};
ExpressionInfoVisitor(expression_info).visit(moving_predicate);
/// TODO: If there is no group by, where, and prewhere expression, we can push down the stateful function
if (expression_info.is_stateful_function)
return false;
if (expression_info.is_aggregate_function)
having_predicates.emplace_back(moving_predicate);
else
where_predicates.emplace_back(moving_predicate);
}
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery,
ASTSelectQuery::Expression expr)
if (having_predicates.empty())
select_query.setExpression(ASTSelectQuery::Expression::HAVING, {});
else
{
ASTPtr subquery_expression = subquery->getExpression(expr, false);
subquery_expression = subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression;
auto having_predicate = reduce_predicates(having_predicates);
select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_predicate));
}
if (!where_predicates.empty())
{
auto moved_predicate = reduce_predicates(where_predicates);
moved_predicate = select_query.where() ? makeASTFunction("and", select_query.where(), moved_predicate) : moved_predicate;
select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(moved_predicate));
}
subquery->setExpression(expr, std::move(subquery_expression));
return true;
}
PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns()
{
SubqueriesProjectionColumns projection_columns;
for (const auto & table_expression : getTableExpressions(*ast_select))
if (table_expression->subquery)
getSubqueryProjectionColumns(table_expression->subquery, projection_columns);
return projection_columns;
}
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns)
{
String qualified_name_prefix = subquery->tryGetAlias();
if (!qualified_name_prefix.empty())
qualified_name_prefix += '.';
const ASTPtr & subselect = subquery->children[0];
ASTs select_with_union_projections;
const auto * select_with_union_query = subselect->as<ASTSelectWithUnionQuery>();
for (auto & select : select_with_union_query->list_of_selects->children)
{
std::vector<ProjectionWithAlias> subquery_projections;
auto select_projection_columns = getSelectQueryProjectionColumns(select);
if (!select_projection_columns.empty())
{
if (select_with_union_projections.empty())
select_with_union_projections = select_projection_columns;
for (size_t i = 0; i < select_projection_columns.size(); i++)
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
projection_columns.insert(std::pair(select->as<ASTSelectQuery>(), subquery_projections));
}
}
}
ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast)
{
ASTs projection_columns;
auto * select_query = ast->as<ASTSelectQuery>();
/// first should normalize query tree.
std::unordered_map<String, ASTPtr> aliases;
std::vector<DatabaseAndTableWithAlias> tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase());
/// TODO: get tables from evaluateAsterisk instead of tablesOnly() to extract asterisks in general way
std::vector<TableWithColumnNames> tables_with_columns = TranslateQualifiedNamesVisitor::Data::tablesOnly(tables);
TranslateQualifiedNamesVisitor::Data qn_visitor_data({}, std::move(tables_with_columns), false);
TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast);
QueryAliasesVisitor::Data query_aliases_data{aliases};
QueryAliasesVisitor(query_aliases_data).visit(ast);
MarkTableIdentifiersVisitor::Data mark_tables_data{aliases};
MarkTableIdentifiersVisitor(mark_tables_data).visit(ast);
QueryNormalizer::Data normalizer_data(aliases, settings);
QueryNormalizer(normalizer_data).visit(ast);
for (const auto & projection_column : select_query->select()->children)
{
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>() || projection_column->as<ASTColumnsMatcher>())
{
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
for (const auto & column : evaluated_columns)
projection_columns.emplace_back(column);
continue;
}
projection_columns.emplace_back(projection_column);
}
return projection_columns;
}
ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk)
{
/// SELECT *, SELECT dummy, SELECT 1 AS id
if (!select_query->tables() || select_query->tables()->children.empty())
return {};
std::vector<const ASTTableExpression *> tables_expression = getTableExpressions(*select_query);
if (const auto * qualified_asterisk = asterisk->as<ASTQualifiedAsterisk>())
{
if (qualified_asterisk->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
for (auto it = tables_expression.begin(); it != tables_expression.end();)
{
const ASTTableExpression * table_expression = *it;
DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase());
if (ident_db_and_name.satisfies(database_and_table_with_alias, true))
++it;
else
it = tables_expression.erase(it); /// It's not a required table
}
}
ASTs projection_columns;
for (auto & table_expression : tables_expression)
{
if (table_expression->subquery)
{
const auto * subquery = table_expression->subquery->as<ASTSubquery>();
const auto * select_with_union_query = subquery->children[0]->as<ASTSelectWithUnionQuery>();
const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]);
projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end());
}
else
{
StoragePtr storage;
if (table_expression->table_function)
{
auto query_context = const_cast<Context *>(&context.getQueryContext());
storage = query_context->executeTableFunction(table_expression->table_function);
}
else if (table_expression->database_and_table_name)
{
const auto * database_and_table_ast = table_expression->database_and_table_name->as<ASTIdentifier>();
DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast);
storage = context.getTable(database_and_table_name.database, database_and_table_name.table);
}
else
throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR);
const auto block = storage->getSampleBlock();
if (const auto * asterisk_pattern = asterisk->as<ASTColumnsMatcher>())
{
for (size_t idx = 0; idx < block.columns(); ++idx)
{
auto & col = block.getByPosition(idx);
if (asterisk_pattern->isColumnMatching(col.name))
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(col.name));
}
}
else
{
for (size_t idx = 0; idx < block.columns(); ++idx)
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
}
}
}
return projection_columns;
}
void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression)
{
const auto my_alias = expression->tryGetAlias();
if (!my_alias.empty())
expression->setAlias("");
for (auto & child : expression->children)
cleanExpressionAlias(child);
}
}

View File

@ -1,110 +1,53 @@
#pragma once
#include "DatabaseAndTableWithAlias.h"
#include <Parsers/ASTSelectQuery.h>
#include <map>
#include <Interpreters/DatabaseAndTableWithAlias.h>
namespace DB
{
class ASTIdentifier;
class ASTSubquery;
class Context;
struct Settings;
/** This class provides functions for Push-Down predicate expressions
*
* The Example:
* - Query before optimization :
* SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a UNION ALL SELECT id_2, name_2 FROM table_b)
* WHERE id_1 = 1
* - Query after optimization :
* SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a WHERE id_1 = 1 UNION ALL SELECT id_2, name_2 FROM table_b WHERE id_2 = 1)
* WHERE id_1 = 1
/** Predicate optimization based on rewriting ast rules
* For more details : https://github.com/ClickHouse/ClickHouse/pull/2015#issuecomment-374283452
* The optimizer does two different optimizations
* - Move predicates from having to where
* - Push the predicate down from the current query to the having of the subquery
*/
class PredicateExpressionsOptimizer
{
using ProjectionWithAlias = std::pair<ASTPtr, String>;
using SubqueriesProjectionColumns = std::map<ASTSelectQuery *, std::vector<ProjectionWithAlias>>;
using IdentifierWithQualifier = std::pair<ASTIdentifier *, String>;
public:
PredicateExpressionsOptimizer(const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_);
bool optimize(ASTSelectQuery & select_query);
private:
/// Extracts settings, mostly to show which are used and which are not.
struct ExtractedSettings
{
/// QueryNormalizer settings
const UInt64 max_ast_depth;
const UInt64 max_expanded_ast_elements;
const String count_distinct_implementation;
/// for PredicateExpressionsOptimizer
const bool enable_optimize_predicate_expression;
const bool enable_optimize_predicate_expression_to_final_subquery;
const bool join_use_nulls;
template<typename T>
ExtractedSettings(const T & settings_)
: max_ast_depth(settings_.max_ast_depth),
max_expanded_ast_elements(settings_.max_expanded_ast_elements),
count_distinct_implementation(settings_.count_distinct_implementation),
enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression),
enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery),
join_use_nulls(settings_.join_use_nulls)
: enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression),
enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery)
{}
};
public:
PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_);
bool optimize();
private:
ASTSelectQuery * ast_select;
const ExtractedSettings settings;
const Context & context;
const std::vector<TableWithColumnNames> & tables_with_columns;
enum OptimizeKind
{
NONE,
PUSH_TO_PREWHERE,
PUSH_TO_WHERE,
PUSH_TO_HAVING,
};
const ExtractedSettings settings;
bool isArrayJoinFunction(const ASTPtr & node);
std::vector<ASTs> extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere);
std::vector<ASTPtr> splitConjunctionPredicate(const ASTPtr & predicate_expression);
bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector<ASTs> & tables_predicates);
std::vector<IdentifierWithQualifier> getDependenciesAndQualifiers(ASTPtr & expression,
std::vector<TableWithColumnNames> & tables_with_aliases);
bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const;
bool optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, ASTSelectQuery::Expression expr);
bool optimizeImpl(const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
bool allowPushDown(
const ASTSelectQuery * subquery,
const ASTPtr & outer_predicate,
const std::vector<ProjectionWithAlias> & subquery_projection_columns,
const std::vector<IdentifierWithQualifier> & outer_predicate_dependencies,
OptimizeKind & optimize_kind);
bool checkDependencies(
const std::vector<ProjectionWithAlias> & projection_columns,
const std::vector<IdentifierWithQualifier> & dependencies,
OptimizeKind & optimize_kind);
void setNewAliasesForInnerPredicate(const std::vector<ProjectionWithAlias> & projection_columns,
const std::vector<IdentifierWithQualifier> & inner_predicate_dependencies);
SubqueriesProjectionColumns getAllSubqueryProjectionColumns();
void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns);
ASTs getSelectQueryProjectionColumns(ASTPtr & ast);
ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk);
void cleanExpressionAlias(ASTPtr & expression);
bool tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query);
};
}

View File

@ -0,0 +1,119 @@
#include <Interpreters/PredicateRewriteVisitor.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/ExtractExpressionInfoVisitor.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
PredicateRewriteVisitorData::PredicateRewriteVisitorData(
const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_)
: context(context_), predicates(predicates_), column_names(column_names_), optimize_final(optimize_final_)
{
}
void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &)
{
auto & internal_select_list = union_select_query.list_of_selects->children;
if (internal_select_list.size() > 0)
visitFirstInternalSelect(*internal_select_list[0]->as<ASTSelectQuery>(), internal_select_list[0]);
for (size_t index = 1; index < internal_select_list.size(); ++index)
visitOtherInternalSelect(*internal_select_list[index]->as<ASTSelectQuery>(), internal_select_list[index]);
}
void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &)
{
is_rewrite |= rewriteSubquery(select_query, column_names, column_names);
}
void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &)
{
/// For non first select, its alias has no more significance, so we can set a temporary alias for them
ASTPtr temp_internal_select = select_query.clone();
ASTSelectQuery * temp_select_query = temp_internal_select->as<ASTSelectQuery>();
size_t alias_index = 0;
for (auto & ref_select : temp_select_query->refSelect()->children)
{
if (!ref_select->as<ASTAsterisk>() && !ref_select->as<ASTQualifiedAsterisk>() && !ref_select->as<ASTColumnsMatcher>() &&
!ref_select->as<ASTIdentifier>())
{
if (const auto & alias = ref_select->tryGetAlias(); alias.empty())
ref_select->setAlias("--predicate_optimizer_" + toString(alias_index++));
}
}
const Names & internal_columns = InterpreterSelectQuery(
temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames();
if (rewriteSubquery(*temp_select_query, column_names, internal_columns))
{
is_rewrite |= true;
select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect()));
select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving()));
}
}
static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector<ASTIdentifier *> & identifiers)
{
/// Skip WHERE x in (SELECT ...)
if (!predicate->as<ASTSubquery>())
{
for (auto & children : predicate->children)
cleanAliasAndCollectIdentifiers(children, identifiers);
}
if (const auto alias = predicate->tryGetAlias(); !alias.empty())
predicate->setAlias("");
if (ASTIdentifier * identifier = predicate->as<ASTIdentifier>())
identifiers.emplace_back(identifier);
}
bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns)
{
if ((!optimize_final && subquery.final())
|| subquery.with() || subquery.withFill()
|| subquery.limitBy() || subquery.limitLength()
|| hasStatefulFunction(subquery.select(), context))
return false;
for (const auto & predicate : predicates)
{
std::vector<ASTIdentifier *> identifiers;
ASTPtr optimize_predicate = predicate->clone();
cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers);
for (size_t index = 0; index < identifiers.size(); ++index)
{
const auto & column_name = identifiers[index]->shortName();
const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name);
/// For lambda functions, we can't always find them in the list of columns
/// For example: SELECT * FROM system.one WHERE arrayMap(x -> x, [dummy]) = [0]
if (outer_column_iterator != outer_columns.end())
identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]);
}
/// We only need to push all the predicates to subquery having
/// The subquery optimizer will move the appropriate predicates from having to where
subquery.setExpression(ASTSelectQuery::Expression::HAVING,
subquery.having() ? makeASTFunction("and", optimize_predicate, subquery.having()) : optimize_predicate);
}
return true;
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class PredicateRewriteVisitorData
{
public:
bool is_rewrite = false;
using TypeToVisit = ASTSelectWithUnionQuery;
void visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &);
PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_);
private:
const Context & context;
const ASTs & predicates;
const Names & column_names;
bool optimize_final;
void visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &);
void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &);
bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns);
};
using PredicateRewriteMatcher = OneTypeMatcher<PredicateRewriteVisitorData, false>;
using PredicateRewriteVisitor = InDepthNodeVisitor<PredicateRewriteMatcher, true>;
}

View File

@ -920,6 +920,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
if (select_query)
{
/// Push the predicate expression down to the subqueries.
result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query);
/// GROUP BY injective function elimination.
optimizeGroupBy(select_query, source_columns_set, context);
@ -935,9 +938,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
/// array_join_alias_to_name, array_join_result_to_source.
getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set);
/// Push the predicate expression down to the subqueries.
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys,
result.analyzed_join->table_join);
collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases);

View File

@ -0,0 +1,95 @@
#include <Processors/DelayedPortsProcessor.h>
namespace DB
{
DelayedPortsProcessor::DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports)
: IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header))
, num_delayed(delayed_ports.size())
{
port_pairs.resize(num_ports);
auto input_it = inputs.begin();
auto output_it = outputs.begin();
for (size_t i = 0; i < num_ports; ++i)
{
port_pairs[i].input_port = &*input_it;
port_pairs[i].output_port = &*output_it;
++input_it;
++output_it;
}
for (auto & delayed : delayed_ports)
port_pairs[delayed].is_delayed = true;
}
bool DelayedPortsProcessor::processPair(PortsPair & pair)
{
auto finish = [&]()
{
if (!pair.is_finished)
{
pair.is_finished = true;
++num_finished;
}
};
if (pair.output_port->isFinished())
{
pair.input_port->close();
finish();
return false;
}
if (pair.input_port->isFinished())
{
pair.output_port->finish();
finish();
return false;
}
if (!pair.output_port->canPush())
return false;
pair.input_port->setNeeded();
if (pair.input_port->hasData())
pair.output_port->pushData(pair.input_port->pullData());
return true;
}
IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs)
{
bool skip_delayed = (num_finished + num_delayed) < port_pairs.size();
bool need_data = false;
for (auto & output_number : updated_outputs)
{
if (!skip_delayed || !port_pairs[output_number].is_delayed)
need_data = processPair(port_pairs[output_number]) || need_data;
}
for (auto & input_number : updated_inputs)
{
if (!skip_delayed || !port_pairs[input_number].is_delayed)
need_data = processPair(port_pairs[input_number]) || need_data;
}
/// In case if main streams are finished at current iteration, start processing delayed streams.
if (skip_delayed && (num_finished + num_delayed) >= port_pairs.size())
{
for (auto & pair : port_pairs)
if (pair.is_delayed)
need_data = processPair(pair) || need_data;
}
if (num_finished == port_pairs.size())
return Status::Finished;
if (need_data)
return Status::NeedData;
return Status::PortFull;
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <Processors/IProcessor.h>
namespace DB
{
/// Processor with N inputs and N outputs. Only moves data from i-th input to i-th output as is.
/// Some ports are delayed. Delayed ports are processed after other outputs are all finished.
/// Data between ports is not mixed. It is important because this processor can be used before MergingSortedTransform.
/// Delayed ports are appeared after joins, when some non-matched data need to be processed at the end.
class DelayedPortsProcessor : public IProcessor
{
public:
DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports);
String getName() const override { return "DelayedPorts"; }
Status prepare(const PortNumbers &, const PortNumbers &) override;
private:
struct PortsPair
{
InputPort * input_port = nullptr;
OutputPort * output_port = nullptr;
bool is_delayed = false;
bool is_finished = false;
};
std::vector<PortsPair> port_pairs;
size_t num_delayed;
size_t num_finished = 0;
bool processPair(PortsPair & pair);
};
}

View File

@ -64,13 +64,6 @@ bool PipelineExecutor::addEdges(UInt64 node)
throwUnknownProcessor(to_proc, cur, true);
UInt64 proc_num = it->second;
for (auto & edge : edges)
{
if (edge.to == proc_num)
throw Exception("Multiple edges are not allowed for the same processors.", ErrorCodes::LOGICAL_ERROR);
}
auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number, update_list);
from_port.setUpdateInfo(&edge.update_info);
@ -177,10 +170,20 @@ void PipelineExecutor::addJob(ExecutionState * execution_state)
execution_state->job = std::move(job);
}
void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
bool PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
{
auto & cur_node = graph[pid];
auto new_processors = cur_node.processor->expandPipeline();
Processors new_processors;
try
{
new_processors = cur_node.processor->expandPipeline();
}
catch (...)
{
cur_node.execution_state->exception = std::current_exception();
return false;
}
for (const auto & processor : new_processors)
{
@ -220,20 +223,22 @@ void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
}
}
}
return true;
}
bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack)
bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number)
{
/// In this method we have ownership on edge, but node can be concurrently accessed.
auto & node = graph[edge.to];
std::lock_guard guard(node.status_mutex);
std::unique_lock lock(node.status_mutex);
ExecStatus status = node.status;
if (status == ExecStatus::Finished)
return false;
return true;
if (edge.backward)
node.updated_output_ports.push_back(edge.output_port_number);
@ -243,14 +248,13 @@ bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stac
if (status == ExecStatus::Idle)
{
node.status = ExecStatus::Preparing;
stack.push(edge.to);
return prepareProcessor(edge.to, thread_number, queue, std::move(lock));
}
return true;
}
return false;
}
bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async)
bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock<std::mutex> node_lock)
{
/// In this method we have ownership on node.
auto & node = graph[pid];
@ -264,14 +268,22 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
{
/// Stopwatch watch;
std::lock_guard guard(node.status_mutex);
std::unique_lock<std::mutex> lock(std::move(node_lock));
auto status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports);
node.updated_input_ports.clear();
node.updated_output_ports.clear();
try
{
node.last_processor_status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports);
}
catch (...)
{
node.execution_state->exception = std::current_exception();
return false;
}
/// node.execution_state->preparation_time_ns += watch.elapsed();
node.last_processor_status = status;
node.updated_input_ports.clear();
node.updated_output_ports.clear();
switch (node.last_processor_status)
{
@ -291,7 +303,8 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
case IProcessor::Status::Ready:
{
node.status = ExecStatus::Executing;
return true;
queue.push(node.execution_state.get());
break;
}
case IProcessor::Status::Async:
{
@ -303,9 +316,7 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
}
case IProcessor::Status::Wait:
{
if (!async)
throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR);
break;
throw Exception("Wait is temporary not supported.", ErrorCodes::LOGICAL_ERROR);
}
case IProcessor::Status::ExpandPipeline:
{
@ -337,18 +348,26 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
if (need_traverse)
{
for (auto & edge : updated_back_edges)
tryAddProcessorToStackIfUpdated(*edge, parents);
for (auto & edge : updated_direct_edges)
tryAddProcessorToStackIfUpdated(*edge, children);
{
if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number))
return false;
}
for (auto & edge : updated_back_edges)
{
if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number))
return false;
}
}
if (need_expand_pipeline)
{
Stack stack;
executor_contexts[thread_number]->task_list.emplace_back(
node.execution_state.get(),
&parents
&stack
);
ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back();
@ -356,20 +375,32 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
while (!expand_pipeline_task.compare_exchange_strong(expected, desired))
{
doExpandPipeline(expected, true);
if (!doExpandPipeline(expected, true))
return false;
expected = nullptr;
}
doExpandPipeline(desired, true);
if (!doExpandPipeline(desired, true))
return false;
/// Add itself back to be prepared again.
children.push(pid);
}
stack.push(pid);
while (!stack.empty())
{
auto item = stack.top();
if (!prepareProcessor(item, thread_number, queue, std::unique_lock<std::mutex>(graph[item].status_mutex)))
return false;
stack.pop();
}
}
void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing)
return true;
}
bool PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing)
{
std::unique_lock lock(task->mutex);
@ -381,16 +412,20 @@ void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processi
return task->num_waiting_processing_threads >= num_processing_executors || expand_pipeline_task != task;
});
bool result = true;
/// After condvar.wait() task may point to trash. Can change it only if it is still in expand_pipeline_task.
if (expand_pipeline_task == task)
{
expandPipeline(*task->stack, task->node_to_expand->processors_id);
result = expandPipeline(*task->stack, task->node_to_expand->processors_id);
expand_pipeline_task = nullptr;
lock.unlock();
task->condvar.notify_all();
}
return result;
}
void PipelineExecutor::cancel()
@ -459,49 +494,31 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
#if !defined(__APPLE__) && !defined(__FreeBSD__)
/// Specify CPU core for thread if can.
/// It may reduce the number of context swithches.
/*
if (num_threads > 1)
{
cpu_set_t cpu_set;
CPU_ZERO(&cpu_set);
CPU_SET(thread_num, &cpu_set);
if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1)
LOG_TRACE(log, "Cannot set affinity for thread " << num_threads);
}
*/
#endif
UInt64 total_time_ns = 0;
UInt64 execution_time_ns = 0;
UInt64 processing_time_ns = 0;
UInt64 wait_time_ns = 0;
// UInt64 total_time_ns = 0;
// UInt64 execution_time_ns = 0;
// UInt64 processing_time_ns = 0;
// UInt64 wait_time_ns = 0;
Stopwatch total_time_watch;
// Stopwatch total_time_watch;
ExecutionState * state = nullptr;
auto prepare_processor = [&](UInt64 pid, Stack & children, Stack & parents)
auto prepare_processor = [&](UInt64 pid, Queue & queue)
{
try
{
return prepareProcessor(pid, children, parents, thread_num, false);
}
catch (...)
{
graph[pid].execution_state->exception = std::current_exception();
if (!prepareProcessor(pid, thread_num, queue, std::unique_lock<std::mutex>(graph[pid].status_mutex)))
finish();
}
return false;
};
using Queue = std::queue<ExecutionState *>;
auto prepare_all_processors = [&](Queue & queue, Stack & stack, Stack & children, Stack & parents)
{
while (!stack.empty() && !finished)
{
auto current_processor = stack.top();
stack.pop();
if (prepare_processor(current_processor, children, parents))
queue.push(graph[current_processor].execution_state.get());
}
};
auto wake_up_executor = [&](size_t executor)
@ -511,63 +528,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
executor_contexts[executor]->condvar.notify_one();
};
auto process_pinned_tasks = [&](Queue & queue)
{
Queue tmp_queue;
struct PinnedTask
{
ExecutionState * task;
size_t thread_num;
};
std::stack<PinnedTask> pinned_tasks;
while (!queue.empty())
{
auto task = queue.front();
queue.pop();
auto stream = task->processor->getStream();
if (stream != IProcessor::NO_STREAM)
pinned_tasks.push({.task = task, .thread_num = stream % num_threads});
else
tmp_queue.push(task);
}
if (!pinned_tasks.empty())
{
std::stack<size_t> threads_to_wake;
{
std::lock_guard lock(task_queue_mutex);
while (!pinned_tasks.empty())
{
auto & pinned_task = pinned_tasks.top();
auto thread = pinned_task.thread_num;
executor_contexts[thread]->pinned_tasks.push(pinned_task.task);
pinned_tasks.pop();
if (threads_queue.has(thread))
{
threads_queue.pop(thread);
threads_to_wake.push(thread);
}
}
}
while (!threads_to_wake.empty())
{
wake_up_executor(threads_to_wake.top());
threads_to_wake.pop();
}
}
queue.swap(tmp_queue);
};
while (!finished)
{
/// First, find any processor to execute.
@ -577,20 +537,11 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
{
std::unique_lock lock(task_queue_mutex);
if (!executor_contexts[thread_num]->pinned_tasks.empty())
{
state = executor_contexts[thread_num]->pinned_tasks.front();
executor_contexts[thread_num]->pinned_tasks.pop();
break;
}
if (!task_queue.empty())
{
state = task_queue.front();
task_queue.pop();
state = task_queue.pop(thread_num);
if (!task_queue.empty() && !threads_queue.empty())
if (!task_queue.empty() && !threads_queue.empty() /*&& task_queue.quota() > threads_queue.size()*/)
{
auto thread_to_wake = threads_queue.pop_any();
lock.unlock();
@ -648,8 +599,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
/// Try to execute neighbour processor.
{
Stack children;
Stack parents;
Queue queue;
++num_processing_executors;
@ -657,36 +606,16 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
doExpandPipeline(task, true);
/// Execute again if can.
if (!prepare_processor(state->processors_id, children, parents))
prepare_processor(state->processors_id, queue);
state = nullptr;
/// Process all neighbours. Children will be on the top of stack, then parents.
prepare_all_processors(queue, children, children, parents);
process_pinned_tasks(queue);
/// Take local task from queue if has one.
if (!state && !queue.empty())
if (!queue.empty())
{
state = queue.front();
queue.pop();
}
prepare_all_processors(queue, parents, parents, parents);
process_pinned_tasks(queue);
/// Take pinned task if has one.
{
std::lock_guard guard(task_queue_mutex);
if (!executor_contexts[thread_num]->pinned_tasks.empty())
{
if (state)
queue.push(state);
state = executor_contexts[thread_num]->pinned_tasks.front();
executor_contexts[thread_num]->pinned_tasks.pop();
}
}
/// Push other tasks to global queue.
if (!queue.empty())
{
@ -694,14 +623,15 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
while (!queue.empty() && !finished)
{
task_queue.push(queue.front());
task_queue.push(queue.front(), thread_num);
queue.pop();
}
if (!threads_queue.empty())
if (!threads_queue.empty() /* && task_queue.quota() > threads_queue.size()*/)
{
auto thread_to_wake = threads_queue.pop_any();
lock.unlock();
wake_up_executor(thread_to_wake);
}
}
@ -715,14 +645,15 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
}
}
total_time_ns = total_time_watch.elapsed();
wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
// total_time_ns = total_time_watch.elapsed();
// wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
/*
LOG_TRACE(log, "Thread finished."
<< " Total time: " << (total_time_ns / 1e9) << " sec."
<< " Execution time: " << (execution_time_ns / 1e9) << " sec."
<< " Processing time: " << (processing_time_ns / 1e9) << " sec."
<< " Wait time: " << (wait_time_ns / 1e9) << "sec.");
*/
}
void PipelineExecutor::executeImpl(size_t num_threads)
@ -730,6 +661,7 @@ void PipelineExecutor::executeImpl(size_t num_threads)
Stack stack;
threads_queue.init(num_threads);
task_queue.init(num_threads);
{
std::lock_guard guard(executor_contexts_mutex);
@ -763,19 +695,31 @@ void PipelineExecutor::executeImpl(size_t num_threads)
{
std::lock_guard lock(task_queue_mutex);
Queue queue;
size_t next_thread = 0;
while (!stack.empty())
{
UInt64 proc = stack.top();
stack.pop();
if (prepareProcessor(proc, stack, stack, 0, false))
prepareProcessor(proc, 0, queue, std::unique_lock<std::mutex>(graph[proc].status_mutex));
while (!queue.empty())
{
auto cur_state = graph[proc].execution_state.get();
task_queue.push(cur_state);
task_queue.push(queue.front(), next_thread);
queue.pop();
++next_thread;
if (next_thread >= num_threads)
next_thread = 0;
}
}
}
if (num_threads > 1)
{
for (size_t i = 0; i < num_threads; ++i)
{
threads.emplace_back([this, thread_group, thread_num = i, num_threads]
@ -799,6 +743,9 @@ void PipelineExecutor::executeImpl(size_t num_threads)
for (auto & thread : threads)
if (thread.joinable())
thread.join();
}
else
executeSingleThread(0, num_threads);
finished_flag = true;
}

View File

@ -84,6 +84,7 @@ private:
IProcessor * processor = nullptr;
UInt64 processors_id = 0;
bool has_quota = false;
/// Counters for profiling.
size_t num_executed_jobs = 0;
@ -117,6 +118,7 @@ private:
execution_state = std::make_unique<ExecutionState>();
execution_state->processor = processor;
execution_state->processors_id = processor_id;
execution_state->has_quota = processor->hasQuota();
}
Node(Node && other) noexcept
@ -132,7 +134,59 @@ private:
using Stack = std::stack<UInt64>;
using TaskQueue = std::queue<ExecutionState *>;
class TaskQueue
{
public:
void init(size_t num_threads) { queues.resize(num_threads); }
void push(ExecutionState * state, size_t thread_num)
{
queues[thread_num].push(state);
++size_;
if (state->has_quota)
++quota_;
}
ExecutionState * pop(size_t thread_num)
{
if (size_ == 0)
throw Exception("TaskQueue is not empty.", ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < queues.size(); ++i)
{
if (!queues[thread_num].empty())
{
ExecutionState * state = queues[thread_num].front();
queues[thread_num].pop();
--size_;
if (state->has_quota)
++quota_;
return state;
}
++thread_num;
if (thread_num >= queues.size())
thread_num = 0;
}
throw Exception("TaskQueue is not empty.", ErrorCodes::LOGICAL_ERROR);
}
size_t size() const { return size_; }
bool empty() const { return size_ == 0; }
size_t quota() const { return quota_; }
private:
using Queue = std::queue<ExecutionState *>;
std::vector<Queue> queues;
size_t size_ = 0;
size_t quota_ = 0;
};
/// Queue with pointers to tasks. Each thread will concurrently read from it until finished flag is set.
/// Stores processors need to be prepared. Preparing status is already set for them.
@ -173,7 +227,7 @@ private:
std::mutex mutex;
bool wake_flag = false;
std::queue<ExecutionState *> pinned_tasks;
/// std::queue<ExecutionState *> pinned_tasks;
};
std::vector<std::unique_ptr<ExecutorContext>> executor_contexts;
@ -186,19 +240,21 @@ private:
/// Graph related methods.
bool addEdges(UInt64 node);
void buildGraph();
void expandPipeline(Stack & stack, UInt64 pid);
bool expandPipeline(Stack & stack, UInt64 pid);
using Queue = std::queue<ExecutionState *>;
/// Pipeline execution related methods.
void addChildlessProcessorsToStack(Stack & stack);
bool tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack);
bool tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number);
static void addJob(ExecutionState * execution_state);
// TODO: void addAsyncJob(UInt64 pid);
/// Prepare processor with pid number.
/// Check parents and children of current processor and push them to stacks if they also need to be prepared.
/// If processor wants to be expanded, ExpandPipelineTask from thread_number's execution context will be used.
bool prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async);
void doExpandPipeline(ExpandPipelineTask * task, bool processing);
bool prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock<std::mutex> node_lock);
bool doExpandPipeline(ExpandPipelineTask * task, bool processing);
void executeImpl(size_t num_threads);
void executeSingleThread(size_t thread_num, size_t num_threads);

View File

@ -272,12 +272,17 @@ public:
size_t getStream() const { return stream_number; }
constexpr static size_t NO_STREAM = std::numeric_limits<size_t>::max();
void enableQuota() { has_quota = true; }
bool hasQuota() const { return has_quota; }
private:
std::atomic<bool> is_cancelled{false};
std::string processor_description;
size_t stream_number = NO_STREAM;
bool has_quota = false;
};

View File

@ -115,4 +115,13 @@ void Pipe::pinSources(size_t executor_number)
}
}
void Pipe::enableQuota()
{
for (auto & processor : processors)
{
if (auto * source = dynamic_cast<ISource *>(processor.get()))
source->enableQuota();
}
}
}

View File

@ -42,6 +42,8 @@ public:
/// Set information about preferred executor number for sources.
void pinSources(size_t executor_number);
void enableQuota();
void setTotalsPort(OutputPort * totals_) { totals = totals_; }
OutputPort * getTotalsPort() const { return totals; }

View File

@ -18,6 +18,7 @@
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentThread.h>
#include <Processors/DelayedPortsProcessor.h>
namespace DB
{
@ -165,7 +166,6 @@ void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter)
for (size_t stream_num = 0; stream_num < streams.size(); ++stream_num)
add_transform(streams[stream_num], StreamType::Main, stream_num);
add_transform(delayed_stream_port, StreamType::Main);
add_transform(totals_having_port, StreamType::Totals);
add_transform(extremes_port, StreamType::Extremes);
@ -185,7 +185,6 @@ void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & get
void QueryPipeline::addPipe(Processors pipe)
{
checkInitialized();
concatDelayedStream();
if (pipe.empty())
throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR);
@ -224,41 +223,20 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source)
{
checkInitialized();
if (delayed_stream_port)
throw Exception("QueryPipeline already has stream with non joined data.", ErrorCodes::LOGICAL_ERROR);
checkSource(source, false);
assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline");
delayed_stream_port = &source->getOutputs().front();
IProcessor::PortNumbers delayed_streams = { streams.size() };
streams.emplace_back(&source->getOutputs().front());
processors.emplace_back(std::move(source));
}
void QueryPipeline::concatDelayedStream()
{
if (!delayed_stream_port)
return;
auto resize = std::make_shared<ResizeProcessor>(current_header, getNumMainStreams(), 1);
auto stream = streams.begin();
for (auto & input : resize->getInputs())
connect(**(stream++), input);
auto concat = std::make_shared<ConcatProcessor>(current_header, 2);
connect(resize->getOutputs().front(), concat->getInputs().front());
connect(*delayed_stream_port, concat->getInputs().back());
streams = { &concat->getOutputs().front() };
processors.emplace_back(std::move(resize));
processors.emplace_back(std::move(concat));
delayed_stream_port = nullptr;
auto processor = std::make_shared<DelayedPortsProcessor>(current_header, streams.size(), delayed_streams);
addPipe({ std::move(processor) });
}
void QueryPipeline::resize(size_t num_streams, bool force)
{
checkInitialized();
concatDelayedStream();
if (!force && num_streams == getNumStreams())
return;
@ -278,6 +256,12 @@ void QueryPipeline::resize(size_t num_streams, bool force)
processors.emplace_back(std::move(resize));
}
void QueryPipeline::enableQuotaForCurrentStreams()
{
for (auto & stream : streams)
stream->getProcessor().enableQuota();
}
void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform)
{
checkInitialized();
@ -437,7 +421,6 @@ void QueryPipeline::unitePipelines(
std::vector<QueryPipeline> && pipelines, const Block & common_header, const Context & context)
{
checkInitialized();
concatDelayedStream();
addSimpleTransform([&](const Block & header)
{
@ -450,7 +433,6 @@ void QueryPipeline::unitePipelines(
for (auto & pipeline : pipelines)
{
pipeline.checkInitialized();
pipeline.concatDelayedStream();
pipeline.addSimpleTransform([&](const Block & header)
{
@ -490,6 +472,8 @@ void QueryPipeline::unitePipelines(
table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end()));
interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end());
storage_holder.insert(storage_holder.end(), pipeline.storage_holder.begin(), pipeline.storage_holder.end());
max_threads = std::max(max_threads, pipeline.max_threads);
}
if (!extremes.empty())

View File

@ -57,18 +57,19 @@ public:
/// Will read from this stream after all data was read from other streams.
void addDelayedStream(ProcessorPtr source);
bool hasDelayedStream() const { return delayed_stream_port; }
/// Check if resize transform was used. (In that case another distinct transform will be added).
bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); }
void resize(size_t num_streams, bool force = false);
void enableQuotaForCurrentStreams();
void unitePipelines(std::vector<QueryPipeline> && pipelines, const Block & common_header, const Context & context);
PipelineExecutorPtr execute();
size_t getNumStreams() const { return streams.size() + (hasDelayedStream() ? 1 : 0); }
size_t getNumMainStreams() const { return streams.size(); }
size_t getNumStreams() const { return streams.size(); }
bool hasMoreThanOneStream() const { return getNumStreams() > 1; }
bool hasTotals() const { return totals_having_port != nullptr; }
@ -101,9 +102,6 @@ private:
OutputPort * totals_having_port = nullptr;
OutputPort * extremes_port = nullptr;
/// Special port for delayed stream.
OutputPort * delayed_stream_port = nullptr;
/// If resize processor was added to pipeline.
bool has_resize = false;
@ -124,7 +122,6 @@ private:
void checkInitialized();
void checkSource(const ProcessorPtr & source, bool can_have_totals);
void concatDelayedStream();
template <typename TProcessorGetter>
void addSimpleTransformImpl(const TProcessorGetter & getter);

View File

@ -72,15 +72,33 @@ namespace
class ConvertingAggregatedToChunksSource : public ISource
{
public:
static constexpr UInt32 NUM_BUCKETS = 256;
struct SharedData
{
std::atomic<UInt32> next_bucket_to_merge = 0;
std::array<std::atomic<Int32>, NUM_BUCKETS> source_for_bucket;
SharedData()
{
for (auto & source : source_for_bucket)
source = -1;
}
};
using SharedDataPtr = std::shared_ptr<SharedData>;
ConvertingAggregatedToChunksSource(
AggregatingTransformParamsPtr params_,
ManyAggregatedDataVariantsPtr data_,
Arena * arena_,
std::shared_ptr<std::atomic<UInt32>> next_bucket_to_merge_)
SharedDataPtr shared_data_,
Int32 source_number_,
Arena * arena_)
: ISource(params_->getHeader())
, params(std::move(params_))
, data(std::move(data_))
, next_bucket_to_merge(std::move(next_bucket_to_merge_))
, shared_data(std::move(shared_data_))
, source_number(source_number_)
, arena(arena_)
{}
@ -89,23 +107,25 @@ public:
protected:
Chunk generate() override
{
UInt32 bucket_num = next_bucket_to_merge->fetch_add(1);
UInt32 bucket_num = shared_data->next_bucket_to_merge.fetch_add(1);
if (bucket_num >= NUM_BUCKETS)
return {};
Block block = params->aggregator.mergeAndConvertOneBucketToBlock(*data, arena, params->final, bucket_num);
Chunk chunk = convertToChunk(block);
return convertToChunk(block);
shared_data->source_for_bucket[bucket_num] = source_number;
return chunk;
}
private:
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
std::shared_ptr<std::atomic<UInt32>> next_bucket_to_merge;
SharedDataPtr shared_data;
Int32 source_number;
Arena * arena;
static constexpr UInt32 NUM_BUCKETS = 256;
};
/// Generates chunks with aggregated data.
@ -159,6 +179,7 @@ public:
auto & out = source->getOutputs().front();
inputs.emplace_back(out.getHeader(), this);
connect(out, inputs.back());
inputs.back().setNeeded();
}
return std::move(processors);
@ -200,7 +221,7 @@ public:
return Status::Ready;
/// Two-level case.
return preparePullFromInputs();
return prepareTwoLevel();
}
private:
@ -220,38 +241,37 @@ private:
}
/// Read all sources and try to push current bucket.
IProcessor::Status preparePullFromInputs()
IProcessor::Status prepareTwoLevel()
{
bool all_inputs_are_finished = true;
auto & output = outputs.front();
for (auto & input : inputs)
Int32 next_input_num = shared_data->source_for_bucket[current_bucket_num];
if (next_input_num < 0)
return Status::NeedData;
auto next_input = std::next(inputs.begin(), next_input_num);
/// next_input can't be finished till data was not pulled.
if (!next_input->hasData())
return Status::NeedData;
output.push(next_input->pull());
++current_bucket_num;
if (current_bucket_num == NUM_BUCKETS)
{
if (input.isFinished())
continue;
all_inputs_are_finished = false;
input.setNeeded();
if (input.hasData())
ready_chunks.emplace_back(input.pull());
output.finish();
/// Do not close inputs, they must be finished.
return Status::Finished;
}
moveReadyChunksToMap();
if (trySetCurrentChunkFromCurrentBucket())
return preparePushToOutput();
if (all_inputs_are_finished)
throw Exception("All sources have finished before getting enough data in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
return Status::NeedData;
return Status::PortFull;
}
private:
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
ConvertingAggregatedToChunksSource::SharedDataPtr shared_data;
size_t num_threads;
bool is_initialized = false;
@ -259,49 +279,12 @@ private:
bool finished = false;
Chunk current_chunk;
Chunks ready_chunks;
UInt32 current_bucket_num = 0;
static constexpr Int32 NUM_BUCKETS = 256;
std::map<UInt32, Chunk> bucket_to_chunk;
Processors processors;
static Int32 getBucketFromChunk(const Chunk & chunk)
{
auto & info = chunk.getChunkInfo();
if (!info)
throw Exception("Chunk info was not set for chunk in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(info.get());
if (!agg_info)
throw Exception("Chunk should have AggregatedChunkInfo in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
return agg_info->bucket_num;
}
void moveReadyChunksToMap()
{
for (auto & chunk : ready_chunks)
{
auto bucket = getBucketFromChunk(chunk);
if (bucket < 0 || bucket >= NUM_BUCKETS)
throw Exception("Invalid bucket number " + toString(bucket) + " in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
if (bucket_to_chunk.count(bucket))
throw Exception("Found several chunks with the same bucket number in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
bucket_to_chunk[bucket] = std::move(chunk);
}
ready_chunks.clear();
}
void setCurrentChunk(Chunk chunk)
{
if (has_input)
@ -366,34 +349,17 @@ private:
void createSources()
{
AggregatedDataVariantsPtr & first = data->at(0);
auto next_bucket_to_merge = std::make_shared<std::atomic<UInt32>>(0);
shared_data = std::make_shared<ConvertingAggregatedToChunksSource::SharedData>();
for (size_t thread = 0; thread < num_threads; ++thread)
{
Arena * arena = first->aggregates_pools.at(thread).get();
auto source = std::make_shared<ConvertingAggregatedToChunksSource>(
params, data, arena, next_bucket_to_merge);
params, data, shared_data, thread, arena);
processors.emplace_back(std::move(source));
}
}
bool trySetCurrentChunkFromCurrentBucket()
{
auto it = bucket_to_chunk.find(current_bucket_num);
if (it != bucket_to_chunk.end())
{
setCurrentChunk(std::move(it->second));
++current_bucket_num;
if (current_bucket_num == NUM_BUCKETS)
finished = true;
return true;
}
return false;
}
};
AggregatingTransform::AggregatingTransform(Block header, AggregatingTransformParamsPtr params_)

View File

@ -1,4 +1,4 @@
/* iopyright (c) 2018 BlackBerry Limited
/* Copyright (c) 2018 BlackBerry Limited
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
@ -95,6 +95,66 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name,
DB::ErrorCodes::LOGICAL_ERROR);
}
MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(const Context & context)
{
ASTPtr mergeable_query = inner_query;
if (inner_subquery)
mergeable_query = inner_subquery;
MergeableBlocksPtr new_mergeable_blocks = std::make_shared<MergeableBlocks>();
BlocksPtrs new_blocks = std::make_shared<std::vector<BlocksPtr>>();
BlocksPtr base_blocks = std::make_shared<Blocks>();
InterpreterSelectQuery interpreter(mergeable_query->clone(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names());
auto view_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(interpreter.execute().in);
while (Block this_block = view_mergeable_stream->read())
base_blocks->push_back(this_block);
new_blocks->push_back(base_blocks);
new_mergeable_blocks->blocks = new_blocks;
new_mergeable_blocks->sample_block = view_mergeable_stream->getHeader();
return new_mergeable_blocks;
}
BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks, Block & sample_block)
{
BlockInputStreams streams;
for (auto & blocks_ : *blocks)
{
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
streams.push_back(std::move(stream));
}
return streams;
}
/// Complete query using input streams from mergeable blocks
BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from)
{
auto block_context = std::make_unique<Context>(global_context);
block_context->makeQueryContext();
auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, parent_storage->getColumns(),
std::move(from), QueryProcessingStage::WithMergeableState);
block_context->addExternalTable(table_name + "_blocks", blocks_storage);
InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete));
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
data = std::make_shared<SquashingBlockInputStream>(
data, global_context.getSettingsRef().min_insert_block_size_rows,
global_context.getSettingsRef().min_insert_block_size_bytes);
return data;
}
void StorageLiveView::writeIntoLiveView(
StorageLiveView & live_view,
@ -102,8 +162,6 @@ void StorageLiveView::writeIntoLiveView(
const Context & context)
{
BlockOutputStreamPtr output = std::make_shared<LiveViewBlockOutputStream>(live_view);
auto block_context = std::make_unique<Context>(context.getGlobalContext());
block_context->makeQueryContext();
/// Check if live view has any readers if not
/// just reset blocks to empty and do nothing else
@ -119,54 +177,40 @@ void StorageLiveView::writeIntoLiveView(
bool is_block_processed = false;
BlockInputStreams from;
BlocksPtrs mergeable_blocks;
MergeableBlocksPtr mergeable_blocks;
BlocksPtr new_mergeable_blocks = std::make_shared<Blocks>();
ASTPtr mergeable_query = live_view.getInnerQuery();
if (live_view.getInnerSubQuery())
mergeable_query = live_view.getInnerSubQuery();
{
std::lock_guard lock(live_view.mutex);
mergeable_blocks = live_view.getMergeableBlocks();
if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh)
if (!mergeable_blocks || mergeable_blocks->blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh)
{
mergeable_blocks = std::make_shared<std::vector<BlocksPtr>>();
BlocksPtr base_mergeable_blocks = std::make_shared<Blocks>();
InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names());
auto view_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
interpreter.execute().in);
while (Block this_block = view_mergeable_stream->read())
base_mergeable_blocks->push_back(this_block);
mergeable_blocks->push_back(base_mergeable_blocks);
mergeable_blocks = live_view.collectMergeableBlocks(context);
live_view.setMergeableBlocks(mergeable_blocks);
/// Create from streams
for (auto & blocks_ : *mergeable_blocks)
{
if (blocks_->empty())
continue;
auto sample_block = blocks_->front().cloneEmpty();
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
from.push_back(std::move(stream));
}
from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block);
is_block_processed = true;
}
}
auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName());
if (!is_block_processed)
{
ASTPtr mergeable_query = live_view.getInnerQuery();
if (live_view.getInnerSubQuery())
mergeable_query = live_view.getInnerSubQuery();
BlockInputStreams streams = {std::make_shared<OneBlockInputStream>(block)};
auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name,
parent_storage->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns);
live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns);
InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage,
QueryProcessingStage::WithMergeableState);
auto data_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
select_block.execute().in);
while (Block this_block = data_mergeable_stream->read())
new_mergeable_blocks->push_back(this_block);
@ -177,32 +221,12 @@ void StorageLiveView::writeIntoLiveView(
std::lock_guard lock(live_view.mutex);
mergeable_blocks = live_view.getMergeableBlocks();
mergeable_blocks->push_back(new_mergeable_blocks);
/// Create from streams
for (auto & blocks_ : *mergeable_blocks)
{
if (blocks_->empty())
continue;
auto sample_block = blocks_->front().cloneEmpty();
BlockInputStreamPtr stream = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(blocks_), sample_block);
from.push_back(std::move(stream));
}
mergeable_blocks->blocks->push_back(new_mergeable_blocks);
from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block);
}
}
auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, parent_storage->getColumns(), std::move(from), QueryProcessingStage::WithMergeableState);
block_context->addExternalTable(live_view.table_name + "_blocks", blocks_storage);
InterpreterSelectQuery select(live_view.getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete));
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
data = std::make_shared<SquashingBlockInputStream>(
data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes);
BlockInputStreamPtr data = live_view.completeQuery(from);
copyData(*data, *output);
}
@ -247,6 +271,8 @@ StorageLiveView::StorageLiveView(
DatabaseAndTableName(select_database_name, select_table_name),
DatabaseAndTableName(database_name, table_name));
parent_storage = local_context.getTable(select_database_name, select_table_name);
is_temporary = query.temporary;
temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds();
@ -298,36 +324,10 @@ bool StorageLiveView::getNewBlocks()
UInt128 key;
BlocksPtr new_blocks = std::make_shared<Blocks>();
BlocksMetadataPtr new_blocks_metadata = std::make_shared<BlocksMetadata>();
BlocksPtr new_mergeable_blocks = std::make_shared<Blocks>();
ASTPtr mergeable_query = inner_query;
if (inner_subquery)
mergeable_query = inner_subquery;
InterpreterSelectQuery interpreter(mergeable_query->clone(), *live_view_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names());
auto mergeable_stream = std::make_shared<MaterializingBlockInputStream>(interpreter.execute().in);
while (Block block = mergeable_stream->read())
new_mergeable_blocks->push_back(block);
auto block_context = std::make_unique<Context>(global_context);
block_context->makeQueryContext();
mergeable_blocks = std::make_shared<std::vector<BlocksPtr>>();
mergeable_blocks->push_back(new_mergeable_blocks);
BlockInputStreamPtr from = std::make_shared<BlocksBlockInputStream>(std::make_shared<BlocksPtr>(new_mergeable_blocks), mergeable_stream->getHeader());
auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, global_context.getTable(select_database_name, select_table_name)->getColumns(), {from}, QueryProcessingStage::WithMergeableState);
block_context->addExternalTable(table_name + "_blocks", blocks_storage);
InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete));
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
data = std::make_shared<SquashingBlockInputStream>(
data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes);
mergeable_blocks = collectMergeableBlocks(*live_view_context);
BlockInputStreams from = blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block);
BlockInputStreamPtr data = completeQuery({from});
while (Block block = data->read())
{

View File

@ -27,9 +27,16 @@ struct BlocksMetadata
UInt64 version;
};
struct MergeableBlocks
{
BlocksPtrs blocks;
Block sample_block;
};
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
using BlocksMetadataPtr = std::shared_ptr<BlocksMetadata>;
using MergeableBlocksPtr = std::shared_ptr<MergeableBlocks>;
class StorageLiveView : public ext::shared_ptr_helper<StorageLiveView>, public IStorage
{
@ -45,6 +52,7 @@ public:
String getDatabaseName() const override { return database_name; }
String getSelectDatabaseName() const { return select_database_name; }
String getSelectTableName() const { return select_table_name; }
StoragePtr getParentStorage() const { return parent_storage; }
NameAndTypePair getColumn(const String & column_name) const override;
bool hasColumn(const String & column_name) const override;
@ -138,8 +146,14 @@ public:
unsigned num_streams) override;
std::shared_ptr<BlocksPtr> getBlocksPtr() { return blocks_ptr; }
BlocksPtrs getMergeableBlocks() { return mergeable_blocks; }
void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; }
MergeableBlocksPtr getMergeableBlocks() { return mergeable_blocks; }
/// Collect mergeable blocks and their sample. Must be called holding mutex
MergeableBlocksPtr collectMergeableBlocks(const Context & context);
/// Complete query using input streams from mergeable blocks
BlockInputStreamPtr completeQuery(BlockInputStreams from);
void setMergeableBlocks(MergeableBlocksPtr blocks) { mergeable_blocks = blocks; }
std::shared_ptr<bool> getActivePtr() { return active_ptr; }
/// Read new data blocks that store query result
@ -147,6 +161,9 @@ public:
Block getHeader() const;
/// convert blocks to input streams
static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks, Block & sample_block);
static void writeIntoLiveView(
StorageLiveView & live_view,
const Block & block,
@ -162,6 +179,7 @@ private:
ASTPtr inner_blocks_query; /// query over the mergeable blocks to produce final result
Context & global_context;
std::unique_ptr<Context> live_view_context;
StoragePtr parent_storage;
bool is_temporary = false;
/// Mutex to protect access to sample block
@ -180,7 +198,7 @@ private:
std::shared_ptr<BlocksPtr> blocks_ptr;
/// Current data blocks metadata
std::shared_ptr<BlocksMetadataPtr> blocks_metadata_ptr;
BlocksPtrs mergeable_blocks;
MergeableBlocksPtr mergeable_blocks;
/// Background thread for temporary tables
/// which drops this table if there are no users

View File

@ -169,9 +169,12 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation()
const auto data_settings = data.getSettings();
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
/// DataPart can be store only at one disk. Get Max of free space at all disks
UInt64 disk_space = data.storage_policy->getMaxUnreservedFreeSpace();
/// Allow mutations only if there are enough threads, leave free threads for merges else
if (background_pool_size - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_mutation)
return static_cast<UInt64>(data.storage_policy->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_RESERVE);
return static_cast<UInt64>(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE);
return 0;
}

View File

@ -636,15 +636,16 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size
{
if (isASCII(data[*pos]) && !isAlphaNumericASCII(data[*pos]))
{
/// Finish current token if any
if (*token_len > 0)
return true;
*token_start = ++*pos;
}
else
{
const size_t sz = UTF8::seqLength(static_cast<UInt8>(data[*pos]));
*pos += sz;
*token_len += sz;
/// Note that UTF-8 sequence is completely consisted of non-ASCII bytes.
++*pos;
++*token_len;
}
}
return *token_len > 0;

View File

@ -612,7 +612,13 @@ bool StorageMergeTree::merge(
if (!selected)
{
if (out_disable_reason)
*out_disable_reason = "Cannot select parts for optimization";
{
if (!out_disable_reason->empty())
{
*out_disable_reason += ". ";
}
*out_disable_reason += "Cannot select parts for optimization";
}
return false;
}
@ -697,9 +703,6 @@ bool StorageMergeTree::tryMutatePart()
/// You must call destructor with unlocked `currently_processing_in_background_mutex`.
std::optional<CurrentlyMergingPartsTagger> tagger;
{
/// DataPart can be store only at one disk. Get Max of free space at all disks
UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace();
std::lock_guard lock(currently_processing_in_background_mutex);
if (current_mutations_by_version.empty())
@ -715,7 +718,7 @@ bool StorageMergeTree::tryMutatePart()
if (mutations_begin_it == mutations_end_it)
continue;
if (merger_mutator.getMaxSourcePartSizeForMutation() > disk_space)
if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk)
continue;
size_t current_ast_elements = 0;

View File

@ -3181,7 +3181,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
{
/// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock.
for (auto & merge_entry : merge_entries)
waitForAllReplicasToProcessLogEntry(merge_entry);
waitForAllReplicasToProcessLogEntry(merge_entry, false);
}
return true;
@ -3889,13 +3889,19 @@ StorageReplicatedMergeTree::allocateBlockNumber(
}
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry)
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
{
LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);
Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
auto zookeeper = getZooKeeper();
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
for (const String & replica : replicas)
{
if (wait_for_non_active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
{
waitForReplicaToProcessLogEntry(replica, entry);
}
}
LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
}

View File

@ -486,7 +486,7 @@ private:
* Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock.
* TODO: There are wrong usages of this method that are not fixed yet.
*/
void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry);
void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
/** Wait until the specified replica executes the specified action from the log.
* NOTE: See comment about locks above.

View File

@ -1,6 +1,7 @@
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/getTableExpressions.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSubquery.h>
@ -23,6 +24,7 @@ namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR;
extern const int ALIAS_REQUIRED;
}
@ -62,9 +64,24 @@ BlockInputStreams StorageView::read(
replaceTableNameWithSubquery(new_outer_select, new_inner_query);
if (PredicateExpressionsOptimizer(new_outer_select, context.getSettings(), context).optimize())
/// TODO: remove getTableExpressions and getTablesWithColumns
{
const auto & table_expressions = getTableExpressions(*new_outer_select);
const auto & tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context);
auto & settings = context.getSettingsRef();
if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1)
{
for (auto & pr : tables_with_columns)
if (pr.table.table.empty() && pr.table.alias.empty())
throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).",
ErrorCodes::ALIAS_REQUIRED);
}
if (PredicateExpressionsOptimizer(context, tables_with_columns, context.getSettings()).optimize(*new_outer_select))
current_inner_query = new_inner_query;
}
}
QueryPipeline pipeline;
/// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return

View File

@ -58,6 +58,7 @@ def test_smoke():
assert instance.query("SELECT a FROM mydb.filtered_table1") == "1\n1\n"
assert instance.query("SELECT b FROM mydb.filtered_table1") == "0\n1\n"
assert instance.query("SELECT a FROM mydb.filtered_table1 WHERE a = 1") == "1\n1\n"
assert instance.query("SELECT a FROM mydb.filtered_table1 WHERE a IN (1)") == "1\n1\n"
assert instance.query("SELECT a = 1 FROM mydb.filtered_table1") == "1\n1\n"
assert instance.query("SELECT a FROM mydb.filtered_table3") == "0\n1\n"

View File

@ -37,14 +37,20 @@
<value>rnd</value>
</values>
</substitution>
<substitution>
<name>num_rows</name>
<values>
<value>10000000</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
<query>INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<drop_query>DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec}</drop_query>

View File

@ -37,18 +37,24 @@
<value>rnd</value>
</values>
</substitution>
<substitution>
<name>num_rows</name>
<values>
<value>10000000</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
<fill_query>INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<!-- INSERTs above will be run unspecified amount of times, hence size of table is unknown.
To make test more reliable, we SELECT fixed number of rows. -->
<query>SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1</query>
<query>SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1</query>
<drop_query>DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec}</drop_query>

View File

@ -39,14 +39,20 @@
<value>rnd</value>
</values>
</substitution>
<substitution>
<name>num_rows</name>
<values>
<value>10000000</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
<query>INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
<query>INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<drop_query>DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec}</drop_query>

View File

@ -39,18 +39,24 @@
<value>rnd</value>
</values>
</substitution>
<substitution>
<name>num_rows</name>
<values>
<value>10000000</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
<fill_query>INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<!-- INSERTs above will be run unspecified amount of times, hence size of table is unknown.
To make test more reliable, we SELECT fixed number of rows. -->
<query>SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1</query>
<query>SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1</query>
<drop_query>DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec}</drop_query>

View File

@ -13,7 +13,7 @@ SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b
-------Need push down-------
SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n)
1
SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `2`\n WHERE 0\n)\nWHERE id = 1
SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `--predicate_optimizer_0`\n WHERE 0\n)\nWHERE id = 1
1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1
1

View File

@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_to_kill' ASYNC" &>/dev/nul
sleep 1
# Kill $query_for_pending SYNC. This query is not blocker, so it should be killed fast.
timeout 10 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null
timeout 20 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null
# Both queries have to be killed, doesn't matter with SYNC or ASYNC kill
for run in {1..15}

View File

@ -1,8 +1,8 @@
lickhous
lickhous
lickhous
lickhous
lickhous
lickhous
lickhous
lickhous
lickhou
lickhou
lickhou
lickhou
lickhou
lickhou
lickhou
lickhou

View File

@ -0,0 +1,28 @@
SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0
a 1 0 0
a 2 1 0
a 3 1 0
b 13 2 0
b 15 2 0
SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2)
1 0 3 1
1 0 0 1
SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\'
enable_optimize_predicate_expression
1 val11 val21 val31
SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELECT ccc\n FROM \n (\n SELECT 2 AS ccc\n )\n ANY INNER JOIN \n (\n SELECT 2 AS ccc\n ) USING (ccc)\n WHERE ccc > 1\n)\nWHERE ccc > 1
2
SELECT \n ts, \n id, \n id_b, \n b.ts, \n b.id, \n id_c\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\')
SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n b.ts AS `--b.ts`, \n b.id AS `--b.id`, \n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\')
2 3
3 4
4 5
5 0
2 4
4 0
2 3
4 5
SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1]
0
SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6]
1 2 3

View File

@ -0,0 +1,75 @@
SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 1;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485
ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0;
SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0;
-- https://github.com/ClickHouse/ClickHouse/issues/5682
ANALYZE SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2;
SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2;
-- https://github.com/ClickHouse/ClickHouse/issues/6734
ANALYZE SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression';
SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression';
-- https://github.com/ClickHouse/ClickHouse/issues/6767
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
DROP TABLE IF EXISTS view1;
CREATE TABLE t1 (id UInt32, value1 String ) ENGINE ReplacingMergeTree() ORDER BY id;
CREATE TABLE t2 (id UInt32, value2 String ) ENGINE ReplacingMergeTree() ORDER BY id;
CREATE TABLE t3 (id UInt32, value3 String ) ENGINE ReplacingMergeTree() ORDER BY id;
INSERT INTO t1 (id, value1) VALUES (1, 'val11');
INSERT INTO t2 (id, value2) VALUES (1, 'val21');
INSERT INTO t3 (id, value3) VALUES (1, 'val31');
CREATE VIEW IF NOT EXISTS view1 AS SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3 FROM t1 LEFT JOIN t2 ON t1.id = t2.id LEFT JOIN t3 ON t1.id = t3.id WHERE t1.id > 0;
SELECT * FROM view1 WHERE id = 1;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
DROP TABLE IF EXISTS view1;
-- https://github.com/ClickHouse/ClickHouse/issues/7136
ANALYZE SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1;
SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1;
-- https://github.com/ClickHouse/ClickHouse/issues/5674
-- https://github.com/ClickHouse/ClickHouse/issues/4731
-- https://github.com/ClickHouse/ClickHouse/issues/4904
DROP TABLE IF EXISTS A;
DROP TABLE IF EXISTS B;
CREATE TABLE A (ts DateTime, id String, id_b String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id);
CREATE TABLE B (ts DateTime, id String, id_c String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id);
ANALYZE SELECT ts, id, id_b, b.ts, b.id, id_c FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON b.id = a.id_b WHERE a.ts <= toDateTime('1970-01-01 03:00:00');
ANALYZE SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, b.ts AS `--b.ts`, b.id AS `--b.id`, id_c AS `--b.id_c` FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` WHERE `--a.ts` <= toDateTime('1970-01-01 03:00:00');
DROP TABLE IF EXISTS A;
DROP TABLE IF EXISTS B;
-- https://github.com/ClickHouse/ClickHouse/issues/7802
DROP TABLE IF EXISTS test;
CREATE TABLE test ( A Int32, B Int32 ) ENGINE = Memory();
INSERT INTO test VALUES(1, 2)(0, 3)(1, 4)(0, 5);
SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B);
SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B) WHERE A == 1;
SELECT B, next_B FROM (SELECT A, B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B)) WHERE A == 1;
DROP TABLE IF EXISTS test;
ANALYZE SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1];
SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1];
ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6];
SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6];

View File

@ -0,0 +1,27 @@
bcdef
bcdef
bcdef
bcdef
-
bcdef
bcdef
bcdef
bcdef
-
bcdef
23456
bcdef
3456
bcdef
2345
bcdef
345
-
bcdef
23456
bcdef
3456
bcdef
2345
bcdef
345

View File

@ -0,0 +1,36 @@
select substring('abcdefgh', 2, -2);
select substring('abcdefgh', materialize(2), -2);
select substring('abcdefgh', 2, materialize(-2));
select substring('abcdefgh', materialize(2), materialize(-2));
select '-';
select substring(cast('abcdefgh' as FixedString(8)), 2, -2);
select substring(cast('abcdefgh' as FixedString(8)), materialize(2), -2);
select substring(cast('abcdefgh' as FixedString(8)), 2, materialize(-2));
select substring(cast('abcdefgh' as FixedString(8)), materialize(2), materialize(-2));
select '-';
drop table if exists t;
create table t (s String, l Int8, r Int8) engine = Memory;
insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3);
select substring(s, 2, -2) from t;
select substring(s, l, -2) from t;
select substring(s, 2, r) from t;
select substring(s, l, r) from t;
select '-';
drop table if exists t;
create table t (s FixedString(8), l Int8, r Int8) engine = Memory;
insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3);
select substring(s, 2, -2) from t;
select substring(s, l, -2) from t;
select substring(s, 2, r) from t;
select substring(s, l, r) from t;
drop table if exists t;

View File

@ -3,8 +3,8 @@
# Provides: clickhouse-server
# Default-Start: 2 3 4 5
# Default-Stop: 0 1 6
# Required-Start:
# Required-Stop:
# Required-Start: $network
# Required-Stop: $network
# Short-Description: Yandex clickhouse-server daemon
### END INIT INFO

View File

@ -1,5 +1,7 @@
[Unit]
Description=ClickHouse Server (analytic DBMS for big data)
Requires=network-online.target
After=network-online.target
[Service]
Type=simple

View File

@ -444,7 +444,7 @@ SAMPLE BY intHash32(UserID)
SETTINGS index_granularity = 8192
```
You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you ant.
You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want.
As we can see, `hits_v1` uses the [basic MergeTree engine](../operations/table_engines/mergetree.md), while the `visits_v1` uses the [Collapsing](../operations/table_engines/collapsingmergetree.md) variant.

View File

@ -74,7 +74,7 @@ $ clickhouse client
ROLE_FAMILY UInt32,
ROLE_CODE UInt32
)
ENGINE = MergeTree()
ENGINE = MergeTree ORDER BY date
```
**3.** Exit from ClickHouse console client:

View File

@ -1,19 +1,66 @@
# Conditional functions
## if(cond, then, else), cond ? operator then : else
## `if` function
Returns `then` if `cond != 0`, or `else` if `cond = 0`.
`cond` must be of type `UInt8`, and `then` and `else` must have the lowest common type.
Syntax: `if(cond, then, else)`
`then` and `else` can be `NULL`
Returns `then` if the `cond` is truthy(greater than zero), otherwise returns `else`.
* `cond` must be of type of `UInt8`, and `then` and `else` must have the lowest common type.
* `then` and `else` can be `NULL`
**Example:**
Take this `LEFT_RIGHT` table:
```sql
SELECT *
FROM LEFT_RIGHT
┌─left─┬─right─┐
│ ᴺᵁᴸᴸ │ 4 │
│ 1 │ 3 │
│ 2 │ 2 │
│ 3 │ 1 │
│ 4 │ ᴺᵁᴸᴸ │
└──────┴───────┘
```
The following query compares `left` and `right` values:
```sql
SELECT
left,
right,
if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller
FROM LEFT_RIGHT
WHERE isNotNull(left) AND isNotNull(right)
┌─left─┬─right─┬─is_smaller──────────────────────────┐
│ 1 │ 3 │ left is smaller than right │
│ 2 │ 2 │ right is greater or equal than left │
│ 3 │ 1 │ right is greater or equal than left │
└──────┴───────┴─────────────────────────────────────┘
```
Note: `NULL` values are not used in this example, check [NULL values in conditionals](#null-values-in-conditionals) section.
## Ternary operator
It works same as `if` function.
Syntax: `cond ? then : else`
Returns `then` if the `cond` is truthy(greater than zero), otherwise returns `else`.
* `cond` must be of type of `UInt8`, and `then` and `else` must have the lowest common type.
* `then` and `else` can be `NULL`
## multiIf
Allows you to write the [CASE](../operators.md#operator_case) operator more compactly in the query.
```sql
multiIf(cond_1, then_1, cond_2, then_2...else)
```
Syntax: `multiIf(cond_1, then_1, cond_2, then_2, ..., else)`
**Parameters:**
@ -29,22 +76,76 @@ The function returns one of the values `then_N` or `else`, depending on the cond
**Example**
Take the table
Again using `LEFT_RIGHT` table.
```text
┌─x─┬────y─┐
│ 1 │ ᴺᵁᴸᴸ │
│ 2 │ 3 │
└───┴──────┘
```sql
SELECT
left,
right,
multiIf(left < right, 'left is smaller', left > right, 'left is greater', left = right, 'Both equal', 'Null value') AS result
FROM LEFT_RIGHT
┌─left─┬─right─┬─result──────────┐
│ ᴺᵁᴸᴸ │ 4 │ Null value │
│ 1 │ 3 │ left is smaller │
│ 2 │ 2 │ Both equal │
│ 3 │ 1 │ left is greater │
│ 4 │ ᴺᵁᴸᴸ │ Null value │
└──────┴───────┴─────────────────┘
```
## Using conditional results directly
Run the query `SELECT multiIf(isNull(y) x, y < 3, y, NULL) FROM t_null`. Result:
Conditionals always result to `0`, `1` or `NULL`. So you can use conditional results directly like this:
```text
┌─multiIf(isNull(y), x, less(y, 3), y, NULL)─┐
│ 1 │
```sql
SELECT left < right AS is_small
FROM LEFT_RIGHT
┌─is_small─┐
│ ᴺᵁᴸᴸ │
└────────────────────────────────────────────┘
│ 1 │
│ 0 │
│ 0 │
│ ᴺᵁᴸᴸ │
└──────────┘
```
## NULL values in conditionals
When `NULL` values are involved in conditionals, the result will also be `NULL`.
```sql
SELECT
NULL < 1,
2 < NULL,
NULL < NULL,
NULL = NULL
┌─less(NULL, 1)─┬─less(2, NULL)─┬─less(NULL, NULL)─┬─equals(NULL, NULL)─┐
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │
└───────────────┴───────────────┴──────────────────┴────────────────────┘
```
So you should construct your queries carefully if the types are `Nullable`.
The following example demonstrates this by failing to add equals condition to `multiIf`.
```sql
SELECT
left,
right,
multiIf(left < right, 'left is smaller', left > right, 'right is smaller', 'Both equal') AS faulty_result
FROM LEFT_RIGHT
┌─left─┬─right─┬─faulty_result────┐
│ ᴺᵁᴸᴸ │ 4 │ Both equal │
│ 1 │ 3 │ left is smaller │
│ 2 │ 2 │ Both equal │
│ 3 │ 1 │ right is smaller │
│ 4 │ ᴺᵁᴸᴸ │ Both equal │
└──────┴───────┴──────────────────┘
```
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/conditional_functions/) <!--hide-->

View File

@ -1,6 +1,27 @@
## Fixed in ClickHouse Release 19.14.3.3, 2019-09-10
### CVE-2019-15024
Аn attacker having write access to ZooKeeper and who is able to run a custom server available from the network where ClickHouse runs, can create a custom-built malicious server that will act as a ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from the malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem.
Credits: Eldar Zaitov of Yandex Information Security Team
### CVE-2019-16535
Аn OOB read, OOB write and integer underflow in decompression algorithms can be used to achieve RCE or DoS via native protocol.
Credits: Eldar Zaitov of Yandex Information Security Team
### CVE-2019-16536
Stack overflow leading to DoS can be triggered by malicious authenticated client.
Credits: Eldar Zaitov of Yandex Information Security Team
## Fixed in ClickHouse Release 19.13.6.1, 2019-09-20
### CVE-2019-18657
Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request.
Credits: [Nikita Tikhomirov](https://github.com/NSTikhomirov)
@ -24,6 +45,7 @@ Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Tea
## Fixed in ClickHouse Release 1.1.54388, 2018-06-28
### CVE-2018-14668
"remote" table function allowed arbitrary symbols in "user", "password" and "default_database" fields which led to Cross Protocol Request Forgery Attacks.
Credits: Andrey Krasichkov of Yandex Information Security Team
@ -31,6 +53,7 @@ Credits: Andrey Krasichkov of Yandex Information Security Team
## Fixed in ClickHouse Release 1.1.54390, 2018-07-06
### CVE-2018-14669
ClickHouse MySQL client had "LOAD DATA LOCAL INFILE" functionality enabled that allowed a malicious MySQL database read arbitrary files from the connected ClickHouse server.
Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Team

View File

@ -74,7 +74,7 @@ $ clickhouse client
ROLE_FAMILY UInt32,
ROLE_CODE UInt32
)
ENGINE = MergeTree()
ENGINE = MergeTree ORDER BY date
```
**3.** Выйдите из клиента ClickHouse:

View File

@ -1,3 +1,23 @@
## Исправлено в релизе 19.14.3.3, 2019-09-10
### CVE-2019-15024
Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные.
Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса
### CVE-2019-16535
Интерфейс декомпрессии позволял совершать OOB чтения и записи данных в памяти, а также переполнение целочисленных переменных, что могло приводить к отказу в обслуживании. Также потенциально могло использоваьтся для удаленного выполнения кода.
Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса
### CVE-2019-16536
Аутентифицированный клиент злоумышленника имел возможность вызвать переполнение стека, что могло привести к отказу в обслуживании.
Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса
## Исправлено в релизе 19.13.6.1 от 20 сентября 2019
### CVE-2019-18657

View File

@ -207,7 +207,7 @@ def get_users_info(pull_requests, commits_info, token, max_retries, retry_timeou
# List of unknown commits -> text description.
def process_unknown_commits(commits, commits_info, users):
pattern = 'Commit: [{}]({})\nAuthor: {}\nMessage: {}'
pattern = u'Commit: [{}]({})\nAuthor: {}\nMessage: {}'
texts = []