mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into fix_metadata_version_creation
This commit is contained in:
commit
127fc9ce3c
@ -338,6 +338,12 @@ echo $previous_release_tag | download_release_packets && echo -e 'Download scrip
|
||||
|| echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv
|
||||
|
||||
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log
|
||||
for table in query_log trace_log
|
||||
do
|
||||
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||:
|
||||
done
|
||||
|
||||
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
|
||||
|
||||
# Check if we cloned previous release repository successfully
|
||||
if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
|
||||
@ -454,6 +460,7 @@ else
|
||||
-e "This engine is deprecated and is not supported in transactions" \
|
||||
-e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \
|
||||
-e "The set of parts restored in place of" \
|
||||
-e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
|
||||
/var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "<Error>" > /test_output/bc_check_error_messages.txt \
|
||||
&& echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|
||||
|| echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
|
||||
@ -496,6 +503,12 @@ else
|
||||
|
||||
# Remove file bc_check_fatal_messages.txt if it's empty
|
||||
[ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
|
||||
|
||||
tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||:
|
||||
for table in query_log trace_log
|
||||
do
|
||||
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||:
|
||||
done
|
||||
fi
|
||||
|
||||
dmesg -T > /test_output/dmesg.log
|
||||
@ -505,17 +518,8 @@ grep -q -F -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e
|
||||
&& echo -e 'OOM in dmesg\tFAIL' >> /test_output/test_results.tsv \
|
||||
|| echo -e 'No OOM in dmesg\tOK' >> /test_output/test_results.tsv
|
||||
|
||||
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
|
||||
mv /var/log/clickhouse-server/stderr.log /test_output/
|
||||
|
||||
# Replace the engine with Ordinary to avoid extra symlinks stuff in artifacts.
|
||||
# (so that clickhouse-local --path can read it w/o extra care).
|
||||
sed -i -e "s/ATTACH DATABASE _ UUID '[^']*'/ATTACH DATABASE system/" -e "s/Atomic/Ordinary/" /var/lib/clickhouse/metadata/system.sql
|
||||
for table in query_log trace_log; do
|
||||
sed -i "s/ATTACH TABLE _ UUID '[^']*'/ATTACH TABLE $table/" /var/lib/clickhouse/metadata/system/${table}.sql
|
||||
tar -chf /test_output/${table}_dump.tar /var/lib/clickhouse/metadata/system.sql /var/lib/clickhouse/metadata/system/${table}.sql /var/lib/clickhouse/data/system/${table} ||:
|
||||
done
|
||||
|
||||
# Write check result into check_status.tsv
|
||||
clickhouse-local --structure "test String, res String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by (lower(test) like '%hung%'), rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv
|
||||
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv
|
||||
|
@ -12,7 +12,16 @@ then
|
||||
DIR="amd64"
|
||||
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
|
||||
then
|
||||
DIR="aarch64"
|
||||
# If the system has >=ARMv8.2 (https://en.wikipedia.org/wiki/AArch64), choose the corresponding build, else fall back to a v8.0
|
||||
# compat build. Unfortunately, the ARM ISA level cannot be read directly, we need to guess from the "features" in /proc/cpuinfo.
|
||||
# Also, the flags in /proc/cpuinfo are named differently than the flags passed to the compiler (cmake/cpu_features.cmake).
|
||||
ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/')
|
||||
if [ "${ARMV82}" ]
|
||||
then
|
||||
DIR="aarch64"
|
||||
else
|
||||
DIR="aarch64v80compat"
|
||||
fi
|
||||
elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ]
|
||||
then
|
||||
DIR="powerpc64le"
|
||||
@ -22,12 +31,6 @@ then
|
||||
if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ]
|
||||
then
|
||||
DIR="freebsd"
|
||||
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
|
||||
then
|
||||
DIR="freebsd-aarch64"
|
||||
elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ]
|
||||
then
|
||||
DIR="freebsd-powerpc64le"
|
||||
fi
|
||||
elif [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
@ -42,7 +45,7 @@ fi
|
||||
|
||||
if [ -z "${DIR}" ]
|
||||
then
|
||||
echo "The '${OS}' operating system with the '${ARCH}' architecture is not supported."
|
||||
echo "Operating system '${OS}' / architecture '${ARCH}' is unsupported."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
|
@ -6,7 +6,7 @@ sidebar_label: VIEW
|
||||
|
||||
# CREATE VIEW
|
||||
|
||||
Creates a new view. Views can be [normal](#normal), [materialized](#materialized), [live](#live-view), and [window](#window-view) (live view and window view are experimental features).
|
||||
Creates a new view. Views can be [normal](#normal-view), [materialized](#materialized-view), [live](#live-view-experimental), and [window](#window-view-experimental) (live view and window view are experimental features).
|
||||
|
||||
## Normal View
|
||||
|
||||
|
@ -22,17 +22,17 @@ ClickHouse позволяет отправить на сервер данные,
|
||||
|
||||
Таких секций может быть несколько - по числу передаваемых таблиц.
|
||||
|
||||
**–external** - маркер начала секции.
|
||||
**–file** - путь к файлу с дампом таблицы, или -, что обозначает stdin.
|
||||
Из stdin может быть считана только одна таблица.
|
||||
- **--external** - маркер начала секции.
|
||||
- **--file** - путь к файлу с дампом таблицы, или `-`, что обозначает `stdin`.
|
||||
Из `stdin` может быть считана только одна таблица.
|
||||
|
||||
Следующие параметры не обязательные:
|
||||
**–name** - имя таблицы. Если не указано - используется _data.
|
||||
**–format** - формат данных в файле. Если не указано - используется TabSeparated.
|
||||
- **--name** - имя таблицы. Если не указано - используется _data.
|
||||
- **--format** - формат данных в файле. Если не указано - используется TabSeparated.
|
||||
|
||||
Должен быть указан один из следующих параметров:
|
||||
**–types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы _1, _2, …
|
||||
**–structure** - структура таблицы, в форме `UserID UInt64`, `URL String`. Определяет имена и типы столбцов.
|
||||
- **--types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы _1, _2, …
|
||||
- **--structure** - структура таблицы, в форме `UserID UInt64`, `URL String`. Определяет имена и типы столбцов.
|
||||
|
||||
Файлы, указанные в file, будут разобраны форматом, указанным в format, с использованием типов данных, указанных в types или structure. Таблица будет загружена на сервер, и доступна там в качестве временной таблицы с именем name.
|
||||
|
||||
|
@ -58,7 +58,9 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes
|
||||
}
|
||||
else
|
||||
{
|
||||
auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path));
|
||||
ReadSettings read_settings{};
|
||||
read_settings.local_fs_method = LocalFSReadMethod::pread;
|
||||
auto src_buf = createReadBufferFromFileBase(file_path, read_settings, fs::file_size(file_path));
|
||||
std::shared_ptr<WriteBuffer> dst_buf;
|
||||
|
||||
/// test mode for integration tests.
|
||||
|
@ -70,7 +70,7 @@
|
||||
#include <Client/InternalTextLogs.h>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <IO/ForkWriteBuffer.h>
|
||||
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
using namespace std::literals;
|
||||
@ -292,7 +292,7 @@ void ClientBase::setupSignalHandler()
|
||||
|
||||
ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const
|
||||
{
|
||||
ParserQuery parser(end, global_context->getSettings().allow_settings_after_format_in_insert);
|
||||
std::unique_ptr<IParserBase> parser;
|
||||
ASTPtr res;
|
||||
|
||||
const auto & settings = global_context->getSettingsRef();
|
||||
@ -301,10 +301,17 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
|
||||
if (!allow_multi_statements)
|
||||
max_length = settings.max_query_size;
|
||||
|
||||
const Dialect & dialect = settings.dialect;
|
||||
|
||||
if (dialect == Dialect::kusto)
|
||||
parser = std::make_unique<ParserKQLStatement>(end, global_context->getSettings().allow_settings_after_format_in_insert);
|
||||
else
|
||||
parser = std::make_unique<ParserQuery>(end, global_context->getSettings().allow_settings_after_format_in_insert);
|
||||
|
||||
if (is_interactive || ignore_error)
|
||||
{
|
||||
String message;
|
||||
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
|
||||
if (!res)
|
||||
{
|
||||
@ -314,7 +321,7 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
|
||||
}
|
||||
else
|
||||
{
|
||||
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
if (is_interactive)
|
||||
|
@ -198,6 +198,7 @@ protected:
|
||||
SharedContextHolder shared_context;
|
||||
ContextMutablePtr global_context;
|
||||
|
||||
/// thread status should be destructed before shared context because it relies on process list.
|
||||
std::optional<ThreadStatus> thread_status;
|
||||
|
||||
ServerConnectionPtr connection;
|
||||
|
@ -7,15 +7,11 @@
|
||||
#include <base/unaligned.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
#include <Core/TypeId.h>
|
||||
#include <base/TypeName.h>
|
||||
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MULTITARGET_CODE
|
||||
# include <immintrin.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -395,124 +391,6 @@ protected:
|
||||
Container data;
|
||||
};
|
||||
|
||||
DECLARE_DEFAULT_CODE(
|
||||
template <typename Container, typename Type>
|
||||
inline void vectorIndexImpl(const Container & data, const PaddedPODArray<Type> & indexes, size_t limit, Container & res_data)
|
||||
{
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
res_data[i] = data[indexes[i]];
|
||||
}
|
||||
);
|
||||
|
||||
DECLARE_AVX512VBMI_SPECIFIC_CODE(
|
||||
template <typename Container, typename Type>
|
||||
inline void vectorIndexImpl(const Container & data, const PaddedPODArray<Type> & indexes, size_t limit, Container & res_data)
|
||||
{
|
||||
static constexpr UInt64 MASK64 = 0xffffffffffffffff;
|
||||
const size_t limit64 = limit & ~63;
|
||||
size_t pos = 0;
|
||||
size_t data_size = data.size();
|
||||
|
||||
auto data_pos = reinterpret_cast<const UInt8 *>(data.data());
|
||||
auto indexes_pos = reinterpret_cast<const UInt8 *>(indexes.data());
|
||||
auto res_pos = reinterpret_cast<UInt8 *>(res_data.data());
|
||||
|
||||
if (data_size <= 64)
|
||||
{
|
||||
/// one single mask load for table size <= 64
|
||||
__mmask64 last_mask = MASK64 >> (64 - data_size);
|
||||
__m512i table1 = _mm512_maskz_loadu_epi8(last_mask, data_pos);
|
||||
|
||||
/// 64 bytes table lookup using one single permutexvar_epi8
|
||||
while (pos < limit64)
|
||||
{
|
||||
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
|
||||
__m512i out = _mm512_permutexvar_epi8(vidx, table1);
|
||||
_mm512_storeu_epi8(res_pos + pos, out);
|
||||
pos += 64;
|
||||
}
|
||||
/// tail handling
|
||||
if (limit > limit64)
|
||||
{
|
||||
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
|
||||
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
|
||||
__m512i out = _mm512_permutexvar_epi8(vidx, table1);
|
||||
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
|
||||
}
|
||||
}
|
||||
else if (data_size <= 128)
|
||||
{
|
||||
/// table size (64, 128] requires 2 zmm load
|
||||
__mmask64 last_mask = MASK64 >> (128 - data_size);
|
||||
__m512i table1 = _mm512_loadu_epi8(data_pos);
|
||||
__m512i table2 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 64);
|
||||
|
||||
/// 128 bytes table lookup using one single permute2xvar_epi8
|
||||
while (pos < limit64)
|
||||
{
|
||||
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
|
||||
__m512i out = _mm512_permutex2var_epi8(table1, vidx, table2);
|
||||
_mm512_storeu_epi8(res_pos + pos, out);
|
||||
pos += 64;
|
||||
}
|
||||
if (limit > limit64)
|
||||
{
|
||||
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
|
||||
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
|
||||
__m512i out = _mm512_permutex2var_epi8(table1, vidx, table2);
|
||||
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (data_size > 256)
|
||||
{
|
||||
/// byte index will not exceed 256 boundary.
|
||||
data_size = 256;
|
||||
}
|
||||
|
||||
__m512i table1 = _mm512_loadu_epi8(data_pos);
|
||||
__m512i table2 = _mm512_loadu_epi8(data_pos + 64);
|
||||
__m512i table3, table4;
|
||||
if (data_size <= 192)
|
||||
{
|
||||
/// only 3 tables need to load if size <= 192
|
||||
__mmask64 last_mask = MASK64 >> (192 - data_size);
|
||||
table3 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 128);
|
||||
table4 = _mm512_setzero_si512();
|
||||
}
|
||||
else
|
||||
{
|
||||
__mmask64 last_mask = MASK64 >> (256 - data_size);
|
||||
table3 = _mm512_loadu_epi8(data_pos + 128);
|
||||
table4 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 192);
|
||||
}
|
||||
|
||||
/// 256 bytes table lookup can use: 2 permute2xvar_epi8 plus 1 blender with MSB
|
||||
while (pos < limit64)
|
||||
{
|
||||
__m512i vidx = _mm512_loadu_epi8(indexes_pos + pos);
|
||||
__m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2);
|
||||
__m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4);
|
||||
__mmask64 msb = _mm512_movepi8_mask(vidx);
|
||||
__m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2);
|
||||
_mm512_storeu_epi8(res_pos + pos, out);
|
||||
pos += 64;
|
||||
}
|
||||
if (limit > limit64)
|
||||
{
|
||||
__mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit);
|
||||
__m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos);
|
||||
__m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2);
|
||||
__m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4);
|
||||
__mmask64 msb = _mm512_movepi8_mask(vidx);
|
||||
__m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2);
|
||||
_mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
template <typename T>
|
||||
template <typename Type>
|
||||
ColumnPtr ColumnVector<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
||||
@ -521,18 +399,8 @@ ColumnPtr ColumnVector<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_
|
||||
|
||||
auto res = this->create(limit);
|
||||
typename Self::Container & res_data = res->getData();
|
||||
#if USE_MULTITARGET_CODE
|
||||
if constexpr (sizeof(T) == 1 && sizeof(Type) == 1)
|
||||
{
|
||||
/// VBMI optimization only applicable for (U)Int8 types
|
||||
if (isArchSupported(TargetArch::AVX512VBMI))
|
||||
{
|
||||
TargetSpecific::AVX512VBMI::vectorIndexImpl<Container, Type>(data, indexes, limit, res_data);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
#endif
|
||||
TargetSpecific::Default::vectorIndexImpl<Container, Type>(data, indexes, limit, res_data);
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
res_data[i] = data[indexes[i]];
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/hex.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -226,6 +226,30 @@ String TracingContext::composeTraceparentHeader() const
|
||||
static_cast<uint8_t>(trace_flags));
|
||||
}
|
||||
|
||||
void TracingContext::deserialize(ReadBuffer & buf)
|
||||
{
|
||||
buf >> this->trace_id
|
||||
>> "\n"
|
||||
>> this->span_id
|
||||
>> "\n"
|
||||
>> this->tracestate
|
||||
>> "\n"
|
||||
>> this->trace_flags
|
||||
>> "\n";
|
||||
}
|
||||
|
||||
void TracingContext::serialize(WriteBuffer & buf) const
|
||||
{
|
||||
buf << this->trace_id
|
||||
<< "\n"
|
||||
<< this->span_id
|
||||
<< "\n"
|
||||
<< this->tracestate
|
||||
<< "\n"
|
||||
<< this->trace_flags
|
||||
<< "\n";
|
||||
}
|
||||
|
||||
const TracingContextOnThread & CurrentContext()
|
||||
{
|
||||
return current_thread_trace_context;
|
||||
|
@ -7,6 +7,8 @@ namespace DB
|
||||
|
||||
struct Settings;
|
||||
class OpenTelemetrySpanLog;
|
||||
class WriteBuffer;
|
||||
class ReadBuffer;
|
||||
|
||||
namespace OpenTelemetry
|
||||
{
|
||||
@ -63,6 +65,9 @@ struct TracingContext
|
||||
{
|
||||
return trace_id != UUID();
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf);
|
||||
void serialize(WriteBuffer & buf) const;
|
||||
};
|
||||
|
||||
/// Tracing context kept on each thread
|
||||
@ -157,5 +162,16 @@ struct SpanHolder : public Span
|
||||
|
||||
}
|
||||
|
||||
inline WriteBuffer & operator<<(WriteBuffer & buf, const OpenTelemetry::TracingContext & context)
|
||||
{
|
||||
context.serialize(buf);
|
||||
return buf;
|
||||
}
|
||||
|
||||
inline ReadBuffer & operator>> (ReadBuffer & buf, OpenTelemetry::TracingContext & context)
|
||||
{
|
||||
context.deserialize(buf);
|
||||
return buf;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -705,7 +705,7 @@ void KeeperServer::waitInit()
|
||||
|
||||
int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds();
|
||||
if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag.load(); }))
|
||||
throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization");
|
||||
LOG_WARNING(log, "Failed to wait for RAFT initialization in {}ms, will continue in background", timeout);
|
||||
}
|
||||
|
||||
std::vector<int64_t> KeeperServer::getDeadSessions()
|
||||
|
@ -42,6 +42,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
*/
|
||||
|
||||
#define COMMON_SETTINGS(M) \
|
||||
M(Dialect, dialect, Dialect::clickhouse, "Which SQL dialect will be used to parse query", 0)\
|
||||
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
|
||||
M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
|
||||
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \
|
||||
|
@ -158,5 +158,7 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation , ErrorCodes::BAD_ARGUMENTS,
|
||||
{"str", FormatSettings::MsgPackUUIDRepresentation::STR},
|
||||
{"ext", FormatSettings::MsgPackUUIDRepresentation::EXT}})
|
||||
|
||||
|
||||
IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS,
|
||||
{{"clickhouse", Dialect::clickhouse},
|
||||
{"kusto", Dialect::kusto}})
|
||||
}
|
||||
|
@ -183,4 +183,12 @@ DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule)
|
||||
|
||||
DECLARE_SETTING_ENUM_WITH_RENAME(MsgPackUUIDRepresentation, FormatSettings::MsgPackUUIDRepresentation)
|
||||
|
||||
enum class Dialect
|
||||
{
|
||||
clickhouse,
|
||||
kusto,
|
||||
kusto_auto,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(Dialect)
|
||||
}
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
@ -642,6 +643,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex
|
||||
entry.query = queryToString(query);
|
||||
entry.initiator = ddl_worker->getCommonHostID();
|
||||
entry.setSettingsIfRequired(query_context);
|
||||
entry.tracing_context = OpenTelemetry::CurrentContext();
|
||||
String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context);
|
||||
|
||||
Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
|
||||
|
@ -221,6 +221,10 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
||||
/// NOTE Possibly it would be better to execute initial query on the most up-to-date node,
|
||||
/// but it requires more complex logic around /try node.
|
||||
|
||||
OpenTelemetry::SpanHolder span(__FUNCTION__);
|
||||
span.addAttribute("clickhouse.cluster", database->getDatabaseName());
|
||||
entry.tracing_context = OpenTelemetry::CurrentContext();
|
||||
|
||||
auto zookeeper = getAndSetZooKeeper();
|
||||
UInt32 our_log_ptr = getLogPointer();
|
||||
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
|
||||
|
@ -37,7 +37,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
const ReadSettings & settings_,
|
||||
std::shared_ptr<ReadBufferFromRemoteFSGather> impl_,
|
||||
size_t min_bytes_for_seek_)
|
||||
@ -111,7 +111,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadIndirectBufferFromRemot
|
||||
request.ignore = bytes_to_ignore;
|
||||
bytes_to_ignore = 0;
|
||||
}
|
||||
return reader->submit(request);
|
||||
return reader.submit(request);
|
||||
}
|
||||
|
||||
|
||||
|
@ -31,7 +31,7 @@ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
explicit AsynchronousReadIndirectBufferFromRemoteFS(
|
||||
AsynchronousReaderPtr reader_, const ReadSettings & settings_,
|
||||
IAsynchronousReader & reader_, const ReadSettings & settings_,
|
||||
std::shared_ptr<ReadBufferFromRemoteFSGather> impl_,
|
||||
size_t min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
@ -64,7 +64,7 @@ private:
|
||||
|
||||
std::future<IAsynchronousReader::Result> asyncReadInto(char * data, size_t size);
|
||||
|
||||
AsynchronousReaderPtr reader;
|
||||
IAsynchronousReader & reader;
|
||||
|
||||
Int32 priority;
|
||||
|
||||
|
@ -198,31 +198,10 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ThreadPoolReaderPageCacheMiss);
|
||||
|
||||
ThreadGroupStatusPtr running_group;
|
||||
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
|
||||
running_group = CurrentThread::get().getThreadGroup();
|
||||
auto schedule = threadPoolCallbackRunner<Result>(pool, "ThreadPoolRead");
|
||||
|
||||
ContextPtr query_context;
|
||||
if (CurrentThread::isInitialized())
|
||||
query_context = CurrentThread::get().getQueryContext();
|
||||
|
||||
auto task = std::make_shared<std::packaged_task<Result()>>([request, fd, running_group, query_context]
|
||||
return schedule([request, fd]() -> Result
|
||||
{
|
||||
ThreadStatus thread_status;
|
||||
|
||||
SCOPE_EXIT({
|
||||
if (running_group)
|
||||
thread_status.detachQuery();
|
||||
});
|
||||
|
||||
if (running_group)
|
||||
thread_status.attachQuery(running_group);
|
||||
|
||||
if (query_context)
|
||||
thread_status.attachQueryContext(query_context);
|
||||
|
||||
setThreadName("ThreadPoolRead");
|
||||
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
SCOPE_EXIT({
|
||||
watch.stop();
|
||||
@ -260,14 +239,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
|
||||
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read);
|
||||
|
||||
return Result{ .size = bytes_read, .offset = request.ignore };
|
||||
});
|
||||
|
||||
auto future = task->get_future();
|
||||
|
||||
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
|
||||
pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority);
|
||||
|
||||
return future;
|
||||
}, request.priority);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <IO/AsynchronousReader.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -31,8 +32,11 @@ private:
|
||||
|
||||
public:
|
||||
ThreadPoolReader(size_t pool_size, size_t queue_size_);
|
||||
|
||||
std::future<Result> submit(Request request) override;
|
||||
|
||||
void wait() override { pool.wait(); }
|
||||
|
||||
/// pool automatically waits for all tasks in destructor.
|
||||
};
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/config.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
@ -40,33 +39,10 @@ ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queu
|
||||
|
||||
std::future<IAsynchronousReader::Result> ThreadPoolRemoteFSReader::submit(Request request)
|
||||
{
|
||||
ThreadGroupStatusPtr running_group;
|
||||
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
|
||||
running_group = CurrentThread::get().getThreadGroup();
|
||||
auto schedule = threadPoolCallbackRunner<Result>(pool, "VFSRead");
|
||||
|
||||
ContextPtr query_context;
|
||||
if (CurrentThread::isInitialized())
|
||||
query_context = CurrentThread::get().getQueryContext();
|
||||
|
||||
auto task = std::make_shared<std::packaged_task<Result()>>([request, running_group, query_context]
|
||||
return schedule([request]() -> Result
|
||||
{
|
||||
ThreadStatus thread_status;
|
||||
|
||||
SCOPE_EXIT({
|
||||
if (running_group)
|
||||
thread_status.detachQuery();
|
||||
});
|
||||
|
||||
/// To be able to pass ProfileEvents.
|
||||
if (running_group)
|
||||
thread_status.attachQuery(running_group);
|
||||
|
||||
/// Save query context if any, because cache implementation needs it.
|
||||
if (query_context)
|
||||
thread_status.attachQueryContext(query_context);
|
||||
|
||||
setThreadName("VFSRead");
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::Read};
|
||||
auto * remote_fs_fd = assert_cast<RemoteFSFileDescriptor *>(request.descriptor.get());
|
||||
|
||||
@ -80,14 +56,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolRemoteFSReader::submit(Reques
|
||||
ProfileEvents::increment(ProfileEvents::ThreadpoolReaderReadBytes, result.offset ? result.size - result.offset : result.size);
|
||||
|
||||
return Result{ .size = result.size, .offset = result.offset };
|
||||
});
|
||||
|
||||
auto future = task->get_future();
|
||||
|
||||
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
|
||||
pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority);
|
||||
|
||||
return future;
|
||||
}, request.priority);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/AsynchronousReader.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,6 +15,8 @@ public:
|
||||
|
||||
std::future<IAsynchronousReader::Result> submit(Request request) override;
|
||||
|
||||
void wait() override { pool.wait(); }
|
||||
|
||||
private:
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
@ -11,19 +11,20 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr auto DEFAULT_RETRY_NUM = 3;
|
||||
|
||||
WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const String & blob_path_,
|
||||
size_t max_single_part_upload_size_,
|
||||
size_t buf_size_,
|
||||
const WriteSettings & write_settings_,
|
||||
std::optional<std::map<std::string, std::string>> attributes_)
|
||||
const WriteSettings & write_settings_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size_, nullptr, 0)
|
||||
, blob_container_client(blob_container_client_)
|
||||
, log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"))
|
||||
, max_single_part_upload_size(max_single_part_upload_size_)
|
||||
, blob_path(blob_path_)
|
||||
, write_settings(write_settings_)
|
||||
, attributes(attributes_)
|
||||
, blob_container_client(blob_container_client_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -33,63 +34,69 @@ WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage()
|
||||
finalize();
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func, size_t num_tries)
|
||||
{
|
||||
if (attributes.has_value())
|
||||
auto handle_exception = [&](const auto & e, size_t i)
|
||||
{
|
||||
auto blob_client = blob_container_client->GetBlobClient(blob_path);
|
||||
Azure::Storage::Metadata metadata;
|
||||
for (const auto & [key, value] : *attributes)
|
||||
metadata[key] = value;
|
||||
blob_client.SetMetadata(metadata);
|
||||
}
|
||||
if (i == num_tries - 1)
|
||||
throw;
|
||||
|
||||
const size_t max_tries = 3;
|
||||
for (size_t i = 0; i < max_tries; ++i)
|
||||
LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {}", i + 1, blob_path, e.Message);
|
||||
};
|
||||
|
||||
for (size_t i = 0; i < num_tries; ++i)
|
||||
{
|
||||
try
|
||||
{
|
||||
next();
|
||||
func();
|
||||
break;
|
||||
}
|
||||
catch (const Azure::Core::Http::TransportException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
}
|
||||
catch (const Azure::Core::RequestFailedException & e)
|
||||
{
|
||||
if (i == max_tries - 1)
|
||||
throw;
|
||||
LOG_INFO(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"),
|
||||
"Exception caught during finalizing azure storage write at attempt {}: {}", i + 1, e.Message);
|
||||
handle_exception(e, i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
{
|
||||
execWithRetry([this](){ next(); }, DEFAULT_RETRY_NUM);
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::nextImpl()
|
||||
{
|
||||
if (!offset())
|
||||
return;
|
||||
|
||||
auto * buffer_begin = working_buffer.begin();
|
||||
auto len = offset();
|
||||
char * buffer_begin = working_buffer.begin();
|
||||
size_t total_size = offset();
|
||||
|
||||
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
|
||||
|
||||
size_t read = 0;
|
||||
size_t current_size = 0;
|
||||
std::vector<std::string> block_ids;
|
||||
while (read < len)
|
||||
|
||||
while (current_size < total_size)
|
||||
{
|
||||
auto part_len = std::min(len - read, max_single_part_upload_size);
|
||||
size_t part_len = std::min(total_size - current_size, max_single_part_upload_size);
|
||||
const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64));
|
||||
|
||||
auto block_id = getRandomASCIIString(64);
|
||||
block_ids.push_back(block_id);
|
||||
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + current_size), part_len);
|
||||
execWithRetry([&](){ block_blob_client.StageBlock(block_id, tmp_buffer); }, DEFAULT_RETRY_NUM);
|
||||
|
||||
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + read), part_len);
|
||||
block_blob_client.StageBlock(block_id, tmp_buffer);
|
||||
|
||||
read += part_len;
|
||||
current_size += part_len;
|
||||
LOG_TRACE(log, "Staged block (id: {}) of size {} (written {}/{}, blob path: {}).", block_id, part_len, current_size, total_size, blob_path);
|
||||
}
|
||||
|
||||
block_blob_client.CommitBlockList(block_ids);
|
||||
execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, DEFAULT_RETRY_NUM);
|
||||
LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path);
|
||||
|
||||
if (write_settings.remote_throttler)
|
||||
write_settings.remote_throttler->add(read);
|
||||
write_settings.remote_throttler->add(total_size);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,20 +13,25 @@
|
||||
#include <azure/core/io/body_stream.hpp>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
using AzureClientPtr = std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient>;
|
||||
|
||||
WriteBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
AzureClientPtr blob_container_client_,
|
||||
const String & blob_path_,
|
||||
size_t max_single_part_upload_size_,
|
||||
size_t buf_size_,
|
||||
const WriteSettings & write_settings_,
|
||||
std::optional<std::map<std::string, std::string>> attributes_ = {});
|
||||
const WriteSettings & write_settings_);
|
||||
|
||||
~WriteBufferFromAzureBlobStorage() override;
|
||||
|
||||
@ -34,12 +39,15 @@ public:
|
||||
|
||||
private:
|
||||
void finalizeImpl() override;
|
||||
void execWithRetry(std::function<void()> func, size_t num_tries);
|
||||
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
size_t max_single_part_upload_size;
|
||||
const String blob_path;
|
||||
WriteSettings write_settings;
|
||||
std::optional<std::map<std::string, std::string>> attributes;
|
||||
Poco::Logger * log;
|
||||
|
||||
const size_t max_single_part_upload_size;
|
||||
const std::string blob_path;
|
||||
const WriteSettings write_settings;
|
||||
|
||||
AzureClientPtr blob_container_client;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -77,13 +77,21 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(
|
||||
}
|
||||
else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async)
|
||||
{
|
||||
static AsynchronousReaderPtr reader = std::make_shared<SynchronousReader>();
|
||||
auto context = Context::getGlobalContextInstance();
|
||||
if (!context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
|
||||
|
||||
auto & reader = context->getThreadPoolReader(Context::FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER);
|
||||
res = std::make_unique<AsynchronousReadBufferFromFileWithDescriptorsCache>(
|
||||
reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size);
|
||||
}
|
||||
else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool)
|
||||
{
|
||||
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolReader>(16, 1000000);
|
||||
auto context = Context::getGlobalContextInstance();
|
||||
if (!context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
|
||||
|
||||
auto & reader = context->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER);
|
||||
res = std::make_unique<AsynchronousReadBufferFromFileWithDescriptorsCache>(
|
||||
reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size);
|
||||
}
|
||||
|
@ -107,7 +107,7 @@ std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOL
|
||||
|
||||
if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
auto & reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(reader_impl));
|
||||
}
|
||||
else
|
||||
|
@ -11,22 +11,25 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
AsynchronousReaderPtr IObjectStorage::getThreadPoolReader()
|
||||
IAsynchronousReader & IObjectStorage::getThreadPoolReader()
|
||||
{
|
||||
constexpr size_t pool_size = 50;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolRemoteFSReader>(pool_size, queue_size);
|
||||
return reader;
|
||||
auto context = Context::getGlobalContextInstance();
|
||||
if (!context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
|
||||
|
||||
return context->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
|
||||
}
|
||||
|
||||
ThreadPool & IObjectStorage::getThreadPoolWriter()
|
||||
{
|
||||
constexpr size_t pool_size = 100;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static ThreadPool writer(pool_size, pool_size, queue_size);
|
||||
return writer;
|
||||
auto context = Context::getGlobalContextInstance();
|
||||
if (!context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
|
||||
|
||||
return context->getThreadPoolWriter();
|
||||
}
|
||||
|
||||
void IObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
|
||||
|
@ -130,7 +130,7 @@ public:
|
||||
/// Path to directory with objects cache
|
||||
virtual const std::string & getCacheBasePath() const;
|
||||
|
||||
static AsynchronousReaderPtr getThreadPoolReader();
|
||||
static IAsynchronousReader & getThreadPoolReader();
|
||||
|
||||
static ThreadPool & getThreadPoolWriter();
|
||||
|
||||
|
@ -190,7 +190,7 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
auto & reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(s3_impl));
|
||||
}
|
||||
else
|
||||
@ -230,6 +230,8 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files");
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "VFSWrite");
|
||||
|
||||
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
|
||||
client.get(),
|
||||
bucket,
|
||||
@ -237,7 +239,7 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
|
||||
settings_ptr->s3_settings,
|
||||
attributes,
|
||||
buf_size,
|
||||
threadPoolCallbackRunner(getThreadPoolWriter()),
|
||||
std::move(scheduler),
|
||||
disk_write_settings);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(
|
||||
|
@ -168,7 +168,7 @@ std::unique_ptr<ReadBufferFromFileBase> WebObjectStorage::readObject( /// NOLINT
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = IObjectStorage::getThreadPoolReader();
|
||||
auto & reader = IObjectStorage::getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(web_impl), min_bytes_for_seek);
|
||||
}
|
||||
else
|
||||
|
@ -24,7 +24,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
const std::string & file_name_,
|
||||
size_t buf_size,
|
||||
@ -32,7 +32,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
|
||||
char * existing_memory,
|
||||
size_t alignment,
|
||||
std::optional<size_t> file_size_)
|
||||
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_)
|
||||
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, -1, buf_size, existing_memory, alignment, file_size_)
|
||||
, file_name(file_name_)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::FileOpen);
|
||||
@ -58,7 +58,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
|
||||
|
||||
|
||||
AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
int & fd_,
|
||||
const std::string & original_file_name,
|
||||
@ -66,7 +66,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
|
||||
char * existing_memory,
|
||||
size_t alignment,
|
||||
std::optional<size_t> file_size_)
|
||||
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, fd_, buf_size, existing_memory, alignment, file_size_)
|
||||
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, fd_, buf_size, existing_memory, alignment, file_size_)
|
||||
, file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name)
|
||||
{
|
||||
fd_ = -1;
|
||||
@ -105,4 +105,3 @@ AsynchronousReadBufferFromFileWithDescriptorsCache::~AsynchronousReadBufferFromF
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
@ -14,7 +14,7 @@ protected:
|
||||
|
||||
public:
|
||||
explicit AsynchronousReadBufferFromFile(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
const std::string & file_name_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
@ -25,7 +25,7 @@ public:
|
||||
|
||||
/// Use pre-opened file descriptor.
|
||||
explicit AsynchronousReadBufferFromFile(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object.
|
||||
const std::string & original_file_name = {},
|
||||
@ -45,7 +45,6 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Similar to AsynchronousReadBufferFromFile but also transparently shares open file descriptors.
|
||||
*/
|
||||
class AsynchronousReadBufferFromFileWithDescriptorsCache : public AsynchronousReadBufferFromFileDescriptor
|
||||
@ -56,7 +55,7 @@ private:
|
||||
|
||||
public:
|
||||
AsynchronousReadBufferFromFileWithDescriptorsCache(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
const std::string & file_name_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
@ -64,7 +63,7 @@ public:
|
||||
char * existing_memory = nullptr,
|
||||
size_t alignment = 0,
|
||||
std::optional<size_t> file_size_ = std::nullopt)
|
||||
: AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_)
|
||||
: AsynchronousReadBufferFromFileDescriptor(reader_, priority_, -1, buf_size, existing_memory, alignment, file_size_)
|
||||
, file_name(file_name_)
|
||||
{
|
||||
file = OpenedFileCache::instance().get(file_name, flags);
|
||||
@ -80,4 +79,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -54,7 +54,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadBufferFromFileDescripto
|
||||
return std::async(std::launch::deferred, [] { return IAsynchronousReader::Result{.size = 0, .offset = 0}; });
|
||||
}
|
||||
|
||||
return reader->submit(request);
|
||||
return reader.submit(request);
|
||||
}
|
||||
|
||||
|
||||
@ -140,7 +140,7 @@ void AsynchronousReadBufferFromFileDescriptor::finalize()
|
||||
|
||||
|
||||
AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
int fd_,
|
||||
size_t buf_size,
|
||||
@ -148,7 +148,7 @@ AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescript
|
||||
size_t alignment,
|
||||
std::optional<size_t> file_size_)
|
||||
: ReadBufferFromFileBase(buf_size, existing_memory, alignment, file_size_)
|
||||
, reader(std::move(reader_))
|
||||
, reader(reader_)
|
||||
, priority(priority_)
|
||||
, required_alignment(alignment)
|
||||
, fd(fd_)
|
||||
|
@ -16,7 +16,7 @@ namespace DB
|
||||
class AsynchronousReadBufferFromFileDescriptor : public ReadBufferFromFileBase
|
||||
{
|
||||
protected:
|
||||
AsynchronousReaderPtr reader;
|
||||
IAsynchronousReader & reader;
|
||||
Int32 priority;
|
||||
|
||||
Memory<> prefetch_buffer;
|
||||
@ -36,7 +36,7 @@ protected:
|
||||
|
||||
public:
|
||||
AsynchronousReadBufferFromFileDescriptor(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
Int32 priority_,
|
||||
int fd_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <optional>
|
||||
#include <memory>
|
||||
#include <future>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,7 +19,7 @@ namespace DB
|
||||
* For example, this interface may not suffice if you want to serve 10 000 000 of 4 KiB requests per second.
|
||||
* This interface is fairly limited.
|
||||
*/
|
||||
class IAsynchronousReader
|
||||
class IAsynchronousReader : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// For local filesystems, the file descriptor is simply integer
|
||||
@ -68,6 +69,8 @@ public:
|
||||
/// The method can be called concurrently from multiple threads.
|
||||
virtual std::future<Result> submit(Request request) = 0;
|
||||
|
||||
virtual void wait() = 0;
|
||||
|
||||
/// Destructor must wait for all not completed request and ignore the results.
|
||||
/// It may also cancel the requests.
|
||||
virtual ~IAsynchronousReader() = default;
|
||||
|
@ -43,7 +43,7 @@ struct ParallelReadBuffer::ReadWorker
|
||||
};
|
||||
|
||||
ParallelReadBuffer::ParallelReadBuffer(
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers_)
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory_, ThreadPoolCallbackRunner<void> schedule_, size_t max_working_readers_)
|
||||
: SeekableReadBuffer(nullptr, 0)
|
||||
, max_working_readers(max_working_readers_)
|
||||
, schedule(std::move(schedule_))
|
||||
@ -71,7 +71,7 @@ bool ParallelReadBuffer::addReaderToPool()
|
||||
auto worker = read_workers.emplace_back(std::make_shared<ReadWorker>(std::move(reader)));
|
||||
|
||||
++active_working_reader;
|
||||
schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); });
|
||||
schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }, 0);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -33,12 +33,13 @@ public:
|
||||
class ReadBufferFactory : public WithFileSize
|
||||
{
|
||||
public:
|
||||
~ReadBufferFactory() override = default;
|
||||
|
||||
virtual SeekableReadBufferPtr getReader() = 0;
|
||||
virtual ~ReadBufferFactory() override = default;
|
||||
virtual off_t seek(off_t off, int whence) = 0;
|
||||
};
|
||||
|
||||
explicit ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers);
|
||||
ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, ThreadPoolCallbackRunner<void> schedule_, size_t max_working_readers);
|
||||
|
||||
~ParallelReadBuffer() override { finishAndWait(); }
|
||||
|
||||
@ -75,7 +76,7 @@ private:
|
||||
size_t max_working_readers;
|
||||
std::atomic_size_t active_working_reader{0};
|
||||
|
||||
CallbackRunner schedule;
|
||||
ThreadPoolCallbackRunner<void> schedule;
|
||||
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory;
|
||||
|
||||
|
@ -13,7 +13,8 @@ class SynchronousReader final : public IAsynchronousReader
|
||||
{
|
||||
public:
|
||||
std::future<Result> submit(Request request) override;
|
||||
|
||||
void wait() override {}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -74,7 +74,7 @@ WriteBufferFromS3::WriteBufferFromS3(
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
std::optional<std::map<String, String>> object_metadata_,
|
||||
size_t buffer_size_,
|
||||
ScheduleFunc schedule_,
|
||||
ThreadPoolCallbackRunner<void> schedule_,
|
||||
const WriteSettings & write_settings_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
|
||||
, bucket(bucket_)
|
||||
@ -292,7 +292,7 @@ void WriteBufferFromS3::writePart()
|
||||
}
|
||||
|
||||
task_finish_notify();
|
||||
});
|
||||
}, 0);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -442,7 +442,7 @@ void WriteBufferFromS3::makeSinglepartUpload()
|
||||
}
|
||||
|
||||
task_notify_finish();
|
||||
});
|
||||
}, 0);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
|
||||
#include <aws/core/utils/memory/stl/AWSStringStream.h>
|
||||
|
||||
@ -33,7 +34,6 @@ namespace Aws::S3::Model
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ScheduleFunc = std::function<void(std::function<void()>)>;
|
||||
class WriteBufferFromFile;
|
||||
|
||||
/**
|
||||
@ -53,7 +53,7 @@ public:
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
std::optional<std::map<String, String>> object_metadata_ = std::nullopt,
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
ScheduleFunc schedule_ = {},
|
||||
ThreadPoolCallbackRunner<void> schedule_ = {},
|
||||
const WriteSettings & write_settings_ = {});
|
||||
|
||||
~WriteBufferFromS3() override;
|
||||
@ -106,7 +106,7 @@ private:
|
||||
|
||||
/// Following fields are for background uploads in thread pool (if specified).
|
||||
/// We use std::function to avoid dependency of Interpreters
|
||||
const ScheduleFunc schedule;
|
||||
const ThreadPoolCallbackRunner<void> schedule;
|
||||
|
||||
std::unique_ptr<PutObjectTask> put_object_task; /// Does not need protection by mutex because of the logic around is_finished field.
|
||||
std::list<UploadPartTask> TSA_GUARDED_BY(bg_tasks_mutex) upload_object_tasks;
|
||||
|
@ -31,6 +31,9 @@
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <Disks/IO/ThreadPoolReader.h>
|
||||
#include <IO/SynchronousReader.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/ActionLocksManager.h>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
@ -228,6 +231,12 @@ struct ContextSharedPart : boost::noncopyable
|
||||
mutable std::unique_ptr<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
|
||||
mutable std::unique_ptr<BackgroundSchedulePool> message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
|
||||
|
||||
mutable std::unique_ptr<IAsynchronousReader> asynchronous_remote_fs_reader;
|
||||
mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader;
|
||||
mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader;
|
||||
|
||||
mutable std::unique_ptr<ThreadPool> threadpool_writer;
|
||||
|
||||
mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches
|
||||
mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends
|
||||
mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads
|
||||
@ -311,23 +320,76 @@ struct ContextSharedPart : boost::noncopyable
|
||||
|
||||
~ContextSharedPart()
|
||||
{
|
||||
try
|
||||
/// Wait for thread pool for background reads and writes,
|
||||
/// since it may use per-user MemoryTracker which will be destroyed here.
|
||||
if (asynchronous_remote_fs_reader)
|
||||
{
|
||||
/// Wait for thread pool for background writes,
|
||||
/// since it may use per-user MemoryTracker which will be destroyed here.
|
||||
IObjectStorage::getThreadPoolWriter().wait();
|
||||
/// Make sure that threadpool is destructed before this->process_list
|
||||
/// because thread_status, which was created for threads inside threadpool,
|
||||
/// relies on it.
|
||||
if (load_marks_threadpool)
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Desctructing remote fs threadpool reader");
|
||||
asynchronous_remote_fs_reader->wait();
|
||||
asynchronous_remote_fs_reader.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
if (asynchronous_local_fs_reader)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Desctructing local fs threadpool reader");
|
||||
asynchronous_local_fs_reader->wait();
|
||||
asynchronous_local_fs_reader.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
if (synchronous_local_fs_reader)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Desctructing local fs threadpool reader");
|
||||
synchronous_local_fs_reader->wait();
|
||||
synchronous_local_fs_reader.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
if (threadpool_writer)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Desctructing threadpool writer");
|
||||
threadpool_writer->wait();
|
||||
threadpool_writer.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
if (load_marks_threadpool)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Desctructing marks loader");
|
||||
load_marks_threadpool->wait();
|
||||
load_marks_threadpool.reset();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
@ -3378,6 +3440,66 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const
|
||||
return shared->common_executor;
|
||||
}
|
||||
|
||||
IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const
|
||||
{
|
||||
const auto & config = getConfigRef();
|
||||
|
||||
auto lock = getLock();
|
||||
|
||||
switch (type)
|
||||
{
|
||||
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
|
||||
{
|
||||
if (!shared->asynchronous_remote_fs_reader)
|
||||
{
|
||||
auto pool_size = config.getUInt(".threadpool_remote_fs_reader_pool_size", 100);
|
||||
auto queue_size = config.getUInt(".threadpool_remote_fs_reader_queue_size", 1000000);
|
||||
|
||||
shared->asynchronous_remote_fs_reader = std::make_unique<ThreadPoolRemoteFSReader>(pool_size, queue_size);
|
||||
}
|
||||
|
||||
return *shared->asynchronous_remote_fs_reader;
|
||||
}
|
||||
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
|
||||
{
|
||||
if (!shared->asynchronous_local_fs_reader)
|
||||
{
|
||||
auto pool_size = config.getUInt(".threadpool_local_fs_reader_pool_size", 100);
|
||||
auto queue_size = config.getUInt(".threadpool_local_fs_reader_queue_size", 1000000);
|
||||
|
||||
shared->asynchronous_local_fs_reader = std::make_unique<ThreadPoolReader>(pool_size, queue_size);
|
||||
}
|
||||
|
||||
return *shared->asynchronous_local_fs_reader;
|
||||
}
|
||||
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
|
||||
{
|
||||
if (!shared->synchronous_local_fs_reader)
|
||||
{
|
||||
shared->synchronous_local_fs_reader = std::make_unique<SynchronousReader>();
|
||||
}
|
||||
|
||||
return *shared->synchronous_local_fs_reader;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ThreadPool & Context::getThreadPoolWriter() const
|
||||
{
|
||||
const auto & config = getConfigRef();
|
||||
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->threadpool_writer)
|
||||
{
|
||||
auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100);
|
||||
auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000);
|
||||
|
||||
shared->threadpool_writer = std::make_unique<ThreadPool>(pool_size, pool_size, queue_size);
|
||||
}
|
||||
|
||||
return *shared->threadpool_writer;
|
||||
}
|
||||
|
||||
ReadSettings Context::getReadSettings() const
|
||||
{
|
||||
|
@ -1011,6 +1011,17 @@ public:
|
||||
OrdinaryBackgroundExecutorPtr getFetchesExecutor() const;
|
||||
OrdinaryBackgroundExecutorPtr getCommonExecutor() const;
|
||||
|
||||
enum class FilesystemReaderType
|
||||
{
|
||||
SYNCHRONOUS_LOCAL_FS_READER,
|
||||
ASYNCHRONOUS_LOCAL_FS_READER,
|
||||
ASYNCHRONOUS_REMOTE_FS_READER,
|
||||
};
|
||||
|
||||
IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const;
|
||||
|
||||
ThreadPool & getThreadPoolWriter() const;
|
||||
|
||||
/** Get settings for reading from filesystem. */
|
||||
ReadSettings getReadSettings() const;
|
||||
|
||||
|
@ -50,21 +50,26 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const
|
||||
|
||||
void DDLLogEntry::assertVersion() const
|
||||
{
|
||||
constexpr UInt64 max_version = 2;
|
||||
if (version == 0 || max_version < version)
|
||||
if (version == 0
|
||||
/// NORMALIZE_CREATE_ON_INITIATOR_VERSION does not change the entry format, it uses versioin 2, so there shouldn't be such version
|
||||
|| version == NORMALIZE_CREATE_ON_INITIATOR_VERSION
|
||||
|| version > DDL_ENTRY_FORMAT_MAX_VERSION)
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}."
|
||||
"Maximum supported version is {}", version, max_version);
|
||||
"Maximum supported version is {}", version, DDL_ENTRY_FORMAT_MAX_VERSION);
|
||||
}
|
||||
|
||||
void DDLLogEntry::setSettingsIfRequired(ContextPtr context)
|
||||
{
|
||||
version = context->getSettingsRef().distributed_ddl_entry_format_version;
|
||||
if (version <= 0 || version > DDL_ENTRY_FORMAT_MAX_VERSION)
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown distributed_ddl_entry_format_version: {}."
|
||||
"Maximum supported version is {}.", version, DDL_ENTRY_FORMAT_MAX_VERSION);
|
||||
|
||||
/// NORMALIZE_CREATE_ON_INITIATOR_VERSION does not affect entry format in ZooKeeper
|
||||
if (version == NORMALIZE_CREATE_ON_INITIATOR_VERSION)
|
||||
version = SETTINGS_IN_ZK_VERSION;
|
||||
|
||||
if (version == SETTINGS_IN_ZK_VERSION)
|
||||
if (version >= SETTINGS_IN_ZK_VERSION)
|
||||
settings.emplace(context->getSettingsRef().changes());
|
||||
}
|
||||
|
||||
@ -94,6 +99,9 @@ String DDLLogEntry::toString() const
|
||||
wb << "settings: " << serializeAST(ast) << "\n";
|
||||
}
|
||||
|
||||
if (version >= OPENTELEMETRY_ENABLED_VERSION)
|
||||
wb << "tracing: " << this->tracing_context;
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
@ -106,7 +114,7 @@ void DDLLogEntry::parse(const String & data)
|
||||
|
||||
Strings host_id_strings;
|
||||
rb >> "query: " >> escape >> query >> "\n";
|
||||
if (version == 1)
|
||||
if (version == OLDEST_VERSION)
|
||||
{
|
||||
rb >> "hosts: " >> host_id_strings >> "\n";
|
||||
|
||||
@ -115,9 +123,8 @@ void DDLLogEntry::parse(const String & data)
|
||||
else
|
||||
initiator.clear();
|
||||
}
|
||||
else if (version == 2)
|
||||
else if (version >= SETTINGS_IN_ZK_VERSION)
|
||||
{
|
||||
|
||||
if (!rb.eof() && *rb.position() == 'h')
|
||||
rb >> "hosts: " >> host_id_strings >> "\n";
|
||||
if (!rb.eof() && *rb.position() == 'i')
|
||||
@ -134,6 +141,12 @@ void DDLLogEntry::parse(const String & data)
|
||||
}
|
||||
}
|
||||
|
||||
if (version >= OPENTELEMETRY_ENABLED_VERSION)
|
||||
{
|
||||
if (!rb.eof() && *rb.position() == 't')
|
||||
rb >> "tracing: " >> this->tracing_context;
|
||||
}
|
||||
|
||||
assertEOF(rb);
|
||||
|
||||
if (!host_id_strings.empty())
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <filesystem>
|
||||
|
||||
@ -69,12 +70,18 @@ struct DDLLogEntry
|
||||
static constexpr const UInt64 OLDEST_VERSION = 1;
|
||||
static constexpr const UInt64 SETTINGS_IN_ZK_VERSION = 2;
|
||||
static constexpr const UInt64 NORMALIZE_CREATE_ON_INITIATOR_VERSION = 3;
|
||||
static constexpr const UInt64 OPENTELEMETRY_ENABLED_VERSION = 4;
|
||||
/// Add new version here
|
||||
|
||||
/// Remember to update the value below once new version is added
|
||||
static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 4;
|
||||
|
||||
UInt64 version = 1;
|
||||
String query;
|
||||
std::vector<HostID> hosts;
|
||||
String initiator; // optional
|
||||
std::optional<SettingsChanges> settings;
|
||||
OpenTelemetry::TracingContext tracing_context;
|
||||
|
||||
void setSettingsIfRequired(ContextPtr context);
|
||||
String toString() const;
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
@ -515,6 +516,11 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
|
||||
LOG_DEBUG(log, "Processing task {} ({})", task.entry_name, task.entry.query);
|
||||
chassert(!task.completely_processed);
|
||||
|
||||
/// Setup tracing context on current thread for current DDL
|
||||
OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__ ,
|
||||
task.entry.tracing_context,
|
||||
this->context->getOpenTelemetrySpanLog());
|
||||
|
||||
String active_node_path = task.getActiveNodePath();
|
||||
String finished_node_path = task.getFinishedNodePath();
|
||||
|
||||
|
@ -766,6 +766,16 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
|
||||
throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
|
||||
/// Check if _row_exists for lightweight delete column in column_lists for merge tree family.
|
||||
if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree"))
|
||||
{
|
||||
auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
if (search != all_columns.end())
|
||||
throw Exception("Cannot create table with column '" + LightweightDeleteDescription::FILTER_COLUMN.name + "' "
|
||||
"for *MergeTree engines because it is reserved for lightweight delete feature",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
|
||||
/// Check low cardinality types in creating table if it was not allowed in setting
|
||||
|
@ -55,6 +55,8 @@ bool isSupportedAlterType(int type)
|
||||
|
||||
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, const DDLQueryOnClusterParams & params)
|
||||
{
|
||||
OpenTelemetry::SpanHolder span(__FUNCTION__);
|
||||
|
||||
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ON CLUSTER queries inside transactions are not supported");
|
||||
|
||||
@ -88,6 +90,8 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
|
||||
cluster = context->getCluster(query->cluster);
|
||||
}
|
||||
|
||||
span.addAttribute("clickhouse.cluster", query->cluster);
|
||||
|
||||
/// TODO: support per-cluster grant
|
||||
context->checkAccess(AccessType::CLUSTER);
|
||||
|
||||
@ -164,6 +168,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
|
||||
entry.query = queryToString(query_ptr);
|
||||
entry.initiator = ddl_worker.getCommonHostID();
|
||||
entry.setSettingsIfRequired(context);
|
||||
entry.tracing_context = OpenTelemetry::CurrentContext();
|
||||
String node_path = ddl_worker.enqueueQuery(entry);
|
||||
|
||||
return getDistributedDDLStatus(node_path, entry, context);
|
||||
|
@ -73,6 +73,7 @@
|
||||
#include <memory>
|
||||
#include <random>
|
||||
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -392,10 +393,20 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
String query_table;
|
||||
try
|
||||
{
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
if (settings.dialect == Dialect::kusto && !internal)
|
||||
{
|
||||
ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert);
|
||||
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
}
|
||||
else
|
||||
{
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
{
|
||||
|
@ -1,40 +1,44 @@
|
||||
#include "threadPoolCallbackRunner.h"
|
||||
|
||||
#include <Common/scope_guard_safe.h>
|
||||
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <IO/AsynchronousReader.h>
|
||||
#include <future>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool)
|
||||
template <typename Result> ThreadPoolCallbackRunner<Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name)
|
||||
{
|
||||
return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback) mutable
|
||||
return [pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](std::function<Result()> && callback, size_t priority) mutable -> std::future<Result>
|
||||
{
|
||||
pool->scheduleOrThrow(
|
||||
[&, callback = std::move(callback), thread_group]()
|
||||
{
|
||||
auto task = std::make_shared<std::packaged_task<Result()>>([thread_group, thread_name, callback = std::move(callback)]() -> Result
|
||||
{
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
if (thread_group)
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
|
||||
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
|
||||
/// Typically, it may be changes from Process to User.
|
||||
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
|
||||
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
|
||||
/// When, finally, we destroy the thread (and the ThreadStatus),
|
||||
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,
|
||||
/// and by this time user-level memory tracker may be already destroyed.
|
||||
///
|
||||
/// As a work-around, reset memory tracker to total, which is always alive.
|
||||
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
|
||||
});
|
||||
callback();
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
});
|
||||
|
||||
setThreadName(thread_name.data());
|
||||
|
||||
return callback();
|
||||
});
|
||||
|
||||
auto future = task->get_future();
|
||||
|
||||
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
|
||||
pool->scheduleOrThrow([task]{ (*task)(); }, -priority);
|
||||
|
||||
return future;
|
||||
};
|
||||
}
|
||||
|
||||
template ThreadPoolCallbackRunner<void> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
|
||||
template ThreadPoolCallbackRunner<IAsynchronousReader::Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
|
||||
|
||||
}
|
||||
|
@ -1,15 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <future>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously
|
||||
using CallbackRunner = std::function<void(std::function<void()>)>;
|
||||
/// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously.
|
||||
template <typename Result>
|
||||
using ThreadPoolCallbackRunner = std::function<std::future<Result>(std::function<Result()> &&, size_t priority)>;
|
||||
|
||||
/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'
|
||||
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool);
|
||||
/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'.
|
||||
template <typename Result>
|
||||
ThreadPoolCallbackRunner<Result> threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name);
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,7 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
add_headers_and_sources(clickhouse_parsers .)
|
||||
add_headers_and_sources(clickhouse_parsers ./Access)
|
||||
add_headers_and_sources(clickhouse_parsers ./MySQL)
|
||||
add_headers_and_sources(clickhouse_parsers ./Kusto)
|
||||
add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources})
|
||||
target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils)
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -9,26 +9,6 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
class ParserArray : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/** If in parenthesis an expression from one element - returns this element in `node`;
|
||||
* or if there is a SELECT subquery in parenthesis, then this subquery returned in `node`;
|
||||
* otherwise returns `tuple` function from the contents of brackets.
|
||||
*/
|
||||
class ParserParenthesisExpression : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parenthesized expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/** The SELECT subquery is in parenthesis.
|
||||
*/
|
||||
class ParserSubquery : public IParserBase
|
||||
@ -141,36 +121,6 @@ protected:
|
||||
ColumnTransformers allowed_transformers;
|
||||
};
|
||||
|
||||
/** A function, for example, f(x, y + 1, g(z)).
|
||||
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
|
||||
* Or a parametric aggregate function: quantile(0.9)(x + y).
|
||||
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
|
||||
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
|
||||
*/
|
||||
class ParserFunction : public IParserBase
|
||||
{
|
||||
public:
|
||||
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
|
||||
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "function"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
bool allow_function_parameters;
|
||||
bool is_table_function;
|
||||
};
|
||||
|
||||
// A special function parser for view and viewIfPermitted table functions.
|
||||
// It parses an SELECT query as its argument and doesn't support getColumnName().
|
||||
class ParserTableFunctionView : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "function"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
// Allows to make queries like SELECT SUM(<expr>) FILTER(WHERE <cond>) FROM ...
|
||||
class ParserFilterClause : public IParserBase
|
||||
{
|
||||
@ -394,16 +344,6 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
|
||||
*/
|
||||
class ParserExpressionElement : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "element of expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/** An expression element, possibly with an alias, if appropriate.
|
||||
*/
|
||||
class ParserWithOptionalAlias : public IParserBase
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -116,6 +116,36 @@ private:
|
||||
SelectUnionModes union_modes;
|
||||
};
|
||||
|
||||
|
||||
class ParserArray : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/** A function, for example, f(x, y + 1, g(z)).
|
||||
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
|
||||
* Or a parametric aggregate function: quantile(0.9)(x + y).
|
||||
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
|
||||
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
|
||||
*/
|
||||
class ParserFunction : public IParserBase
|
||||
{
|
||||
public:
|
||||
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
|
||||
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "function"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
bool allow_function_parameters;
|
||||
bool is_table_function;
|
||||
};
|
||||
|
||||
|
||||
/** An expression with an infix binary left-associative operator.
|
||||
* For example, a + b - c + d.
|
||||
*/
|
||||
@ -123,31 +153,13 @@ class ParserLeftAssociativeBinaryOperatorList : public IParserBase
|
||||
{
|
||||
private:
|
||||
Operators_t operators;
|
||||
Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } };
|
||||
ParserPtr first_elem_parser;
|
||||
ParserPtr remaining_elem_parser;
|
||||
/// =, !=, <, > ALL (subquery) / ANY (subquery)
|
||||
bool comparison_expression = false;
|
||||
ParserPtr elem_parser;
|
||||
|
||||
public:
|
||||
/** `operators_` - allowed operators and their corresponding functions
|
||||
*/
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_)
|
||||
: operators(operators_), first_elem_parser(std::move(first_elem_parser_))
|
||||
{
|
||||
}
|
||||
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
|
||||
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool comparison_expression_ = false)
|
||||
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
|
||||
first_elem_parser(std::move(first_elem_parser_)), comparison_expression(comparison_expression_)
|
||||
{
|
||||
}
|
||||
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_,
|
||||
ParserPtr && remaining_elem_parser_)
|
||||
: operators(operators_), first_elem_parser(std::move(first_elem_parser_)),
|
||||
remaining_elem_parser(std::move(remaining_elem_parser_))
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && elem_parser_)
|
||||
: operators(operators_), elem_parser(std::move(elem_parser_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -158,295 +170,8 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/** Expression with an infix operator of arbitrary arity.
|
||||
* For example, a AND b AND c AND d.
|
||||
*/
|
||||
class ParserVariableArityOperatorList : public IParserBase
|
||||
class ParserExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * infix;
|
||||
const char * function_name;
|
||||
ParserPtr elem_parser;
|
||||
|
||||
public:
|
||||
ParserVariableArityOperatorList(const char * infix_, const char * function_, ParserPtr && elem_parser_)
|
||||
: infix(infix_), function_name(function_), elem_parser(std::move(elem_parser_))
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "list, delimited by operator of variable arity"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/** An expression with a prefix unary operator.
|
||||
* Example, NOT x.
|
||||
*/
|
||||
class ParserPrefixUnaryOperatorExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
Operators_t operators;
|
||||
ParserPtr elem_parser;
|
||||
|
||||
public:
|
||||
/** `operators_` - allowed operators and their corresponding functions
|
||||
*/
|
||||
ParserPrefixUnaryOperatorExpression(Operators_t operators_, ParserPtr && elem_parser_)
|
||||
: operators(operators_), elem_parser(std::move(elem_parser_))
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "expression with prefix unary operator"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// CAST operator "::". This parser is used if left argument
|
||||
/// of operator cannot be read as simple literal from text of query.
|
||||
/// Example: "[1, 1 + 1, 1 + 2]::Array(UInt8)"
|
||||
class ParserCastExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserPtr elem_parser;
|
||||
|
||||
public:
|
||||
explicit ParserCastExpression(ParserPtr && elem_parser_)
|
||||
: elem_parser(std::move(elem_parser_))
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "CAST expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserArrayElementExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
|
||||
protected:
|
||||
const char * getName() const override{ return "array element expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserTupleElementExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "tuple element expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserUnaryExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserCastExpression>(std::make_unique<ParserTupleElementExpression>())};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "unary expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserMultiplicativeExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserUnaryExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "multiplicative expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
/// DATE operator. "DATE '2001-01-01'" would be parsed as "toDate('2001-01-01')".
|
||||
class ParserDateOperatorExpression : public IParserBase
|
||||
{
|
||||
protected:
|
||||
ParserMultiplicativeExpression next_parser;
|
||||
|
||||
const char * getName() const override { return "DATE operator expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// TIMESTAMP operator. "TIMESTAMP '2001-01-01 12:34:56'" would be parsed as "toDateTime('2001-01-01 12:34:56')".
|
||||
class ParserTimestampOperatorExpression : public IParserBase
|
||||
{
|
||||
protected:
|
||||
ParserDateOperatorExpression next_parser;
|
||||
|
||||
const char * getName() const override { return "TIMESTAMP operator expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// Optional conversion to INTERVAL data type. Example: "INTERVAL x SECOND" parsed as "toIntervalSecond(x)".
|
||||
class ParserIntervalOperatorExpression : public IParserBase
|
||||
{
|
||||
protected:
|
||||
ParserTimestampOperatorExpression next_parser;
|
||||
|
||||
const char * getName() const override { return "INTERVAL operator expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
static bool parseArgumentAndIntervalKind(Pos & pos, ASTPtr & expr, IntervalKind & interval_kind, Expected & expected);
|
||||
};
|
||||
|
||||
class ParserAdditiveExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserIntervalOperatorExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "additive expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserConcatExpression : public IParserBase
|
||||
{
|
||||
ParserVariableArityOperatorList operator_parser {"||", "concat", std::make_unique<ParserAdditiveExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "string concatenation expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserBetweenExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserConcatExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "BETWEEN expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserComparisonExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
static const char * overlapping_operators_to_skip[];
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators,
|
||||
overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>(), true};
|
||||
|
||||
protected:
|
||||
const char * getName() const override{ return "comparison expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
/** Parser for nullity checking with IS (NOT) NULL.
|
||||
*/
|
||||
class ParserNullityChecking : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserComparisonExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "nullity checking"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserLogicalNotExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserNullityChecking>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override{ return "logical-NOT expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserLogicalAndExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserVariableArityOperatorList operator_parser {"AND", "and", std::make_unique<ParserLogicalNotExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "logical-AND expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserLogicalOrExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserVariableArityOperatorList operator_parser {"OR", "or", std::make_unique<ParserLogicalAndExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "logical-OR expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** An expression with ternary operator.
|
||||
* For example, a = 1 ? b + 1 : c * 2.
|
||||
*/
|
||||
class ParserTernaryOperatorExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserLogicalOrExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "expression with ternary operator"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserLambdaExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserTernaryOperatorExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "lambda expression"; }
|
||||
|
||||
@ -457,9 +182,6 @@ protected:
|
||||
// It's used to parse expressions in table function.
|
||||
class ParserTableFunctionExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserLambdaExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "table function expression"; }
|
||||
|
||||
@ -467,13 +189,10 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
using ParserExpression = ParserLambdaExpression;
|
||||
|
||||
|
||||
class ParserExpressionWithOptionalAlias : public IParserBase
|
||||
{
|
||||
public:
|
||||
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function = false);
|
||||
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false);
|
||||
protected:
|
||||
ParserPtr impl;
|
||||
|
||||
|
@ -138,14 +138,28 @@ void IAST::updateTreeHashImpl(SipHash & hash_state) const
|
||||
}
|
||||
|
||||
|
||||
size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const
|
||||
size_t IAST::checkDepthImpl(size_t max_depth) const
|
||||
{
|
||||
size_t res = level + 1;
|
||||
for (const auto & child : children)
|
||||
std::vector<std::pair<ASTPtr, size_t>> stack;
|
||||
stack.reserve(children.size());
|
||||
|
||||
for (const auto & i: children)
|
||||
stack.push_back({i, 1});
|
||||
|
||||
size_t res = 0;
|
||||
|
||||
while (!stack.empty())
|
||||
{
|
||||
if (level >= max_depth)
|
||||
auto top = stack.back();
|
||||
stack.pop_back();
|
||||
|
||||
if (top.second >= max_depth)
|
||||
throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST);
|
||||
res = std::max(res, child->checkDepthImpl(max_depth, level + 1));
|
||||
|
||||
res = std::max(res, top.second);
|
||||
|
||||
for (const auto & i: top.first->children)
|
||||
stack.push_back({i, top.second + 1});
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -92,7 +92,7 @@ public:
|
||||
*/
|
||||
size_t checkDepth(size_t max_depth) const
|
||||
{
|
||||
return checkDepthImpl(max_depth, 0);
|
||||
return checkDepthImpl(max_depth);
|
||||
}
|
||||
|
||||
/** Get total number of tree elements
|
||||
@ -273,7 +273,7 @@ public:
|
||||
static const char * hilite_none;
|
||||
|
||||
private:
|
||||
size_t checkDepthImpl(size_t max_depth, size_t level) const;
|
||||
size_t checkDepthImpl(size_t max_depth) const;
|
||||
|
||||
/** Forward linked list of ASTPtr to delete.
|
||||
* Used in IAST destructor to avoid possible stack overflow.
|
||||
|
26
src/Parsers/Kusto/ParserKQLFilter.cpp
Normal file
26
src/Parsers/Kusto/ParserKQLFilter.cpp
Normal file
@ -0,0 +1,26 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLFilter.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLFilter :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
String expr = getExprFromToken(pos);
|
||||
ASTPtr where_expression;
|
||||
|
||||
Tokens token_filter(expr.c_str(), expr.c_str()+expr.size());
|
||||
IParser::Pos pos_filter(token_filter, pos.max_depth);
|
||||
if (!ParserExpressionWithOptionalAlias(false).parse(pos_filter, where_expression, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLFilter.h
Normal file
16
src/Parsers/Kusto/ParserKQLFilter.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLFilter : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL where"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
29
src/Parsers/Kusto/ParserKQLLimit.cpp
Normal file
29
src/Parsers/Kusto/ParserKQLLimit.cpp
Normal file
@ -0,0 +1,29 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <cstdlib>
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLLimit :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr limit_length;
|
||||
|
||||
auto expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserExpressionWithOptionalAlias(false).parse(new_pos, limit_length, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLLimit.h
Normal file
16
src/Parsers/Kusto/ParserKQLLimit.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLLimit : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL limit"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
359
src/Parsers/Kusto/ParserKQLOperators.cpp
Normal file
359
src/Parsers/Kusto/ParserKQLOperators.cpp
Normal file
@ -0,0 +1,359 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
String KQLOperators::genHasAnyAllOpExpr(std::vector<String> &tokens, IParser::Pos &token_pos,String kql_op, String ch_op)
|
||||
{
|
||||
String new_expr;
|
||||
Expected expected;
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
|
||||
++token_pos;
|
||||
if (!s_lparen.ignore(token_pos, expected))
|
||||
throw Exception("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
auto haystack = tokens.back();
|
||||
|
||||
String logic_op = (kql_op == "has_all") ? " and " : " or ";
|
||||
|
||||
while (!token_pos->isEnd() && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
|
||||
{
|
||||
auto tmp_arg = String(token_pos->begin, token_pos->end);
|
||||
if (token_pos->type == TokenType::Comma)
|
||||
new_expr = new_expr + logic_op;
|
||||
else
|
||||
new_expr = new_expr + ch_op + "(" + haystack + "," + tmp_arg + ")";
|
||||
|
||||
++token_pos;
|
||||
if (token_pos->type == TokenType::ClosingRoundBracket)
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
tokens.pop_back();
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
String KQLOperators::genInOpExpr(IParser::Pos &token_pos, String kql_op, String ch_op)
|
||||
{
|
||||
String new_expr;
|
||||
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
|
||||
ASTPtr select;
|
||||
Expected expected;
|
||||
|
||||
++token_pos;
|
||||
if (!s_lparen.ignore(token_pos, expected))
|
||||
throw Exception("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
--token_pos;
|
||||
--token_pos;
|
||||
return ch_op;
|
||||
|
||||
}
|
||||
|
||||
String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos)
|
||||
{
|
||||
String new_expr, left_wildcards, right_wildcards, left_space, right_space;
|
||||
|
||||
switch (wildcards_pos)
|
||||
{
|
||||
case WildcardsPos::none:
|
||||
break;
|
||||
|
||||
case WildcardsPos::left:
|
||||
left_wildcards ="%";
|
||||
break;
|
||||
|
||||
case WildcardsPos::right:
|
||||
right_wildcards = "%";
|
||||
break;
|
||||
|
||||
case WildcardsPos::both:
|
||||
left_wildcards ="%";
|
||||
right_wildcards = "%";
|
||||
break;
|
||||
}
|
||||
|
||||
switch (space_pos)
|
||||
{
|
||||
case WildcardsPos::none:
|
||||
break;
|
||||
|
||||
case WildcardsPos::left:
|
||||
left_space =" ";
|
||||
break;
|
||||
|
||||
case WildcardsPos::right:
|
||||
right_space = " ";
|
||||
break;
|
||||
|
||||
case WildcardsPos::both:
|
||||
left_space =" ";
|
||||
right_space = " ";
|
||||
break;
|
||||
}
|
||||
|
||||
++token_pos;
|
||||
|
||||
if (!tokens.empty() && ((token_pos)->type == TokenType::StringLiteral || token_pos->type == TokenType::QuotedIdentifier))
|
||||
new_expr = ch_op +"(" + tokens.back() +", '"+left_wildcards + left_space + String(token_pos->begin + 1,token_pos->end - 1) + right_space + right_wildcards + "')";
|
||||
else if (!tokens.empty() && ((token_pos)->type == TokenType::BareWord))
|
||||
{
|
||||
auto tmp_arg = String(token_pos->begin, token_pos->end);
|
||||
new_expr = ch_op +"(" + tokens.back() +", concat('" + left_wildcards + left_space + "', " + tmp_arg +", '"+ right_space + right_wildcards + "'))";
|
||||
}
|
||||
else
|
||||
throw Exception("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
|
||||
tokens.pop_back();
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
{
|
||||
auto begin = pos;
|
||||
|
||||
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
KQLOperatorValue op_value = KQLOperatorValue::none;
|
||||
|
||||
auto token = String(pos->begin,pos->end);
|
||||
|
||||
String op = token;
|
||||
if (token == "!")
|
||||
{
|
||||
++pos;
|
||||
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
|
||||
throw Exception("Invalid negative operator", ErrorCodes::SYNTAX_ERROR);
|
||||
op ="!"+String(pos->begin,pos->end);
|
||||
}
|
||||
else if (token == "matches")
|
||||
{
|
||||
++pos;
|
||||
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin,pos->end) == "regex")
|
||||
op +=" regex";
|
||||
else
|
||||
--pos;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
op = token;
|
||||
}
|
||||
|
||||
++pos;
|
||||
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin,pos->end) == "~")
|
||||
op +="~";
|
||||
else
|
||||
--pos;
|
||||
}
|
||||
else
|
||||
--pos;
|
||||
|
||||
if (KQLOperator.find(op) == KQLOperator.end())
|
||||
{
|
||||
pos = begin;
|
||||
return false;
|
||||
}
|
||||
|
||||
op_value = KQLOperator[op];
|
||||
|
||||
String new_expr;
|
||||
|
||||
if (op_value == KQLOperatorValue::none)
|
||||
tokens.push_back(op);
|
||||
else
|
||||
{
|
||||
auto last_op = tokens.back();
|
||||
auto last_pos = pos;
|
||||
|
||||
switch (op_value)
|
||||
{
|
||||
case KQLOperatorValue::contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "like", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not like", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::equal:
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_equal:
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::equal_cs:
|
||||
new_expr = "==";
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_equal_cs:
|
||||
new_expr = "!=";
|
||||
break;
|
||||
case KQLOperatorValue::has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_all:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_all", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_any:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_any", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasToken", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasToken", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::in_cs:
|
||||
new_expr = genInOpExpr(pos,op,"in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_in_cs:
|
||||
new_expr = genInOpExpr(pos,op,"not in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::in:
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_in:
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::matches_regex:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "match", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
tokens.push_back(new_expr);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
pos = begin;
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
106
src/Parsers/Kusto/ParserKQLOperators.h
Normal file
106
src/Parsers/Kusto/ParserKQLOperators.h
Normal file
@ -0,0 +1,106 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <unordered_map>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KQLOperators
|
||||
{
|
||||
public:
|
||||
bool convert(std::vector<String> &tokens,IParser::Pos &pos);
|
||||
protected:
|
||||
|
||||
enum class WildcardsPos:uint8_t
|
||||
{
|
||||
none,
|
||||
left,
|
||||
right,
|
||||
both
|
||||
};
|
||||
|
||||
enum class KQLOperatorValue : uint16_t
|
||||
{
|
||||
none,
|
||||
contains,
|
||||
not_contains,
|
||||
contains_cs,
|
||||
not_contains_cs,
|
||||
endswith,
|
||||
not_endswith,
|
||||
endswith_cs,
|
||||
not_endswith_cs,
|
||||
equal, //=~
|
||||
not_equal,//!~
|
||||
equal_cs, //=
|
||||
not_equal_cs,//!=
|
||||
has,
|
||||
not_has,
|
||||
has_all,
|
||||
has_any,
|
||||
has_cs,
|
||||
not_has_cs,
|
||||
hasprefix,
|
||||
not_hasprefix,
|
||||
hasprefix_cs,
|
||||
not_hasprefix_cs,
|
||||
hassuffix,
|
||||
not_hassuffix,
|
||||
hassuffix_cs,
|
||||
not_hassuffix_cs,
|
||||
in_cs, //in
|
||||
not_in_cs, //!in
|
||||
in, //in~
|
||||
not_in ,//!in~
|
||||
matches_regex,
|
||||
startswith,
|
||||
not_startswith,
|
||||
startswith_cs,
|
||||
not_startswith_cs,
|
||||
};
|
||||
|
||||
std::unordered_map <String,KQLOperatorValue> KQLOperator =
|
||||
{
|
||||
{"contains" , KQLOperatorValue::contains},
|
||||
{"!contains" , KQLOperatorValue::not_contains},
|
||||
{"contains_cs" , KQLOperatorValue::contains_cs},
|
||||
{"!contains_cs" , KQLOperatorValue::not_contains_cs},
|
||||
{"endswith" , KQLOperatorValue::endswith},
|
||||
{"!endswith" , KQLOperatorValue::not_endswith},
|
||||
{"endswith_cs" , KQLOperatorValue::endswith_cs},
|
||||
{"!endswith_cs" , KQLOperatorValue::not_endswith_cs},
|
||||
{"=~" , KQLOperatorValue::equal},
|
||||
{"!~" , KQLOperatorValue::not_equal},
|
||||
{"==" , KQLOperatorValue::equal_cs},
|
||||
{"!=" , KQLOperatorValue::not_equal_cs},
|
||||
{"has" , KQLOperatorValue::has},
|
||||
{"!has" , KQLOperatorValue::not_has},
|
||||
{"has_all" , KQLOperatorValue::has_all},
|
||||
{"has_any" , KQLOperatorValue::has_any},
|
||||
{"has_cs" , KQLOperatorValue::has_cs},
|
||||
{"!has_cs" , KQLOperatorValue::not_has_cs},
|
||||
{"hasprefix" , KQLOperatorValue::hasprefix},
|
||||
{"!hasprefix" , KQLOperatorValue::not_hasprefix},
|
||||
{"hasprefix_cs" , KQLOperatorValue::hasprefix_cs},
|
||||
{"!hasprefix_cs" , KQLOperatorValue::not_hasprefix_cs},
|
||||
{"hassuffix" , KQLOperatorValue::hassuffix},
|
||||
{"!hassuffix" , KQLOperatorValue::not_hassuffix},
|
||||
{"hassuffix_cs" , KQLOperatorValue::hassuffix_cs},
|
||||
{"!hassuffix_cs" , KQLOperatorValue::not_hassuffix_cs},
|
||||
{"in" , KQLOperatorValue::in_cs},
|
||||
{"!in" , KQLOperatorValue::not_in_cs},
|
||||
{"in~" , KQLOperatorValue::in},
|
||||
{"!in~" , KQLOperatorValue::not_in},
|
||||
{"matches regex" , KQLOperatorValue::matches_regex},
|
||||
{"startswith" , KQLOperatorValue::startswith},
|
||||
{"!startswith" , KQLOperatorValue::not_startswith},
|
||||
{"startswith_cs" , KQLOperatorValue::startswith_cs},
|
||||
{"!startswith_cs" , KQLOperatorValue::not_startswith_cs},
|
||||
};
|
||||
static String genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos = WildcardsPos::none);
|
||||
static String genInOpExpr(IParser::Pos &token_pos,String kql_op, String ch_op);
|
||||
static String genHasAnyAllOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op);
|
||||
};
|
||||
|
||||
}
|
25
src/Parsers/Kusto/ParserKQLProject.cpp
Normal file
25
src/Parsers/Kusto/ParserKQLProject.cpp
Normal file
@ -0,0 +1,25 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLProject :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
String expr;
|
||||
|
||||
expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str()+expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLProject.h
Normal file
16
src/Parsers/Kusto/ParserKQLProject.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLProject : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL project"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
342
src/Parsers/Kusto/ParserKQLQuery.cpp
Normal file
342
src/Parsers/Kusto/ParserKQLQuery.cpp
Normal file
@ -0,0 +1,342 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLTable.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLFilter.h>
|
||||
#include <Parsers/Kusto/ParserKQLSort.h>
|
||||
#include <Parsers/Kusto/ParserKQLSummarize.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <format>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
String ParserKQLBase :: getExprFromToken(const String & text, const uint32_t & max_depth)
|
||||
{
|
||||
Tokens tokens(text.c_str(), text.c_str() + text.size());
|
||||
IParser::Pos pos(tokens, max_depth);
|
||||
|
||||
return getExprFromToken(pos);
|
||||
}
|
||||
|
||||
String ParserKQLBase :: getExprFromPipe(Pos & pos)
|
||||
{
|
||||
uint16_t bracket_count = 0;
|
||||
auto begin = pos;
|
||||
auto end = pos;
|
||||
while (!end->isEnd() && end->type != TokenType::Semicolon)
|
||||
{
|
||||
if (end->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
|
||||
if (end->type == TokenType::OpeningRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if (end->type == TokenType::PipeMark && bracket_count == 0)
|
||||
break;
|
||||
|
||||
++end;
|
||||
}
|
||||
--end;
|
||||
return String(begin->begin, end->end);
|
||||
}
|
||||
|
||||
String ParserKQLBase :: getExprFromToken(Pos & pos)
|
||||
{
|
||||
String res;
|
||||
std::vector<String> tokens;
|
||||
String alias;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
String token = String(pos->begin,pos->end);
|
||||
|
||||
if (token == "=")
|
||||
{
|
||||
++pos;
|
||||
if (String(pos->begin,pos->end) != "~")
|
||||
{
|
||||
alias = tokens.back();
|
||||
tokens.pop_back();
|
||||
}
|
||||
--pos;
|
||||
}
|
||||
else if (!KQLOperators().convert(tokens,pos))
|
||||
{
|
||||
tokens.push_back(token);
|
||||
}
|
||||
|
||||
if (pos->type == TokenType::Comma && !alias.empty())
|
||||
{
|
||||
tokens.pop_back();
|
||||
tokens.push_back("AS");
|
||||
tokens.push_back(alias);
|
||||
tokens.push_back(",");
|
||||
alias.clear();
|
||||
}
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (!alias.empty())
|
||||
{
|
||||
tokens.push_back("AS");
|
||||
tokens.push_back(alias);
|
||||
}
|
||||
|
||||
for (auto const &token : tokens)
|
||||
res = res.empty()? token : res +" " + token;
|
||||
return res;
|
||||
}
|
||||
|
||||
std::unique_ptr<IParserBase> ParserKQLQuery::getOperator(String & op_name)
|
||||
{
|
||||
if (op_name == "filter" || op_name == "where")
|
||||
return std::make_unique<ParserKQLFilter>();
|
||||
else if (op_name == "limit" || op_name == "take")
|
||||
return std::make_unique<ParserKQLLimit>();
|
||||
else if (op_name == "project")
|
||||
return std::make_unique<ParserKQLProject>();
|
||||
else if (op_name == "sort by" || op_name == "order by")
|
||||
return std::make_unique<ParserKQLSort>();
|
||||
else if (op_name == "summarize")
|
||||
return std::make_unique<ParserKQLSummarize>();
|
||||
else if (op_name == "table")
|
||||
return std::make_unique<ParserKQLTable>();
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
struct KQLOperatorDataFlowState
|
||||
{
|
||||
String operator_name;
|
||||
bool need_input;
|
||||
bool gen_output;
|
||||
int8_t backspace_steps; // how many steps to last token of previous pipe
|
||||
};
|
||||
|
||||
auto select_query = std::make_shared<ASTSelectQuery>();
|
||||
node = select_query;
|
||||
ASTPtr tables;
|
||||
|
||||
std::unordered_map<std::string, KQLOperatorDataFlowState> kql_parser =
|
||||
{
|
||||
{ "filter", {"filter", false, false, 3}},
|
||||
{ "where", {"filter", false, false, 3}},
|
||||
{ "limit", {"limit", false, true, 3}},
|
||||
{ "take", {"limit", false, true, 3}},
|
||||
{ "project", {"project", false, false, 3}},
|
||||
{ "sort by", {"order by", false, false, 4}},
|
||||
{ "order by", {"order by", false, false, 4}},
|
||||
{ "table", {"table", false, false, 3}},
|
||||
{ "summarize", {"summarize", true, true, 3}}
|
||||
};
|
||||
|
||||
std::vector<std::pair<String, Pos>> operation_pos;
|
||||
|
||||
String table_name(pos->begin, pos->end);
|
||||
|
||||
operation_pos.push_back(std::make_pair("table", pos));
|
||||
++pos;
|
||||
uint16_t bracket_count = 0;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if (pos->type == TokenType::PipeMark && bracket_count == 0)
|
||||
{
|
||||
++pos;
|
||||
String kql_operator(pos->begin, pos->end);
|
||||
if (kql_operator == "order" || kql_operator == "sort")
|
||||
{
|
||||
++pos;
|
||||
ParserKeyword s_by("by");
|
||||
if (s_by.ignore(pos,expected))
|
||||
{
|
||||
kql_operator = "order by";
|
||||
--pos;
|
||||
}
|
||||
}
|
||||
if (pos->type != TokenType::BareWord || kql_parser.find(kql_operator) == kql_parser.end())
|
||||
return false;
|
||||
++pos;
|
||||
operation_pos.push_back(std::make_pair(kql_operator, pos));
|
||||
}
|
||||
else
|
||||
++pos;
|
||||
}
|
||||
|
||||
auto kql_operator_str = operation_pos.back().first;
|
||||
auto npos = operation_pos.back().second;
|
||||
if (!npos.isValid())
|
||||
return false;
|
||||
|
||||
auto kql_operator_p = getOperator(kql_operator_str);
|
||||
|
||||
if (!kql_operator_p)
|
||||
return false;
|
||||
|
||||
if (operation_pos.size() == 1)
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else if (operation_pos.size() == 2 && operation_pos.front().first == "table")
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
npos = operation_pos.front().second;
|
||||
if (!ParserKQLTable().parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
String project_clause, order_clause, where_clause, limit_clause;
|
||||
auto last_pos = operation_pos.back().second;
|
||||
auto last_op = operation_pos.back().first;
|
||||
|
||||
auto set_main_query_clause =[&](String & op, Pos & op_pos)
|
||||
{
|
||||
auto op_str = ParserKQLBase::getExprFromPipe(op_pos);
|
||||
if (op == "project")
|
||||
project_clause = op_str;
|
||||
else if (op == "where" || op == "filter")
|
||||
where_clause = where_clause.empty() ? std::format("({})", op_str) : where_clause + std::format("AND ({})", op_str);
|
||||
else if (op == "limit" || op == "take")
|
||||
limit_clause = op_str;
|
||||
else if (op == "order by" || op == "sort by")
|
||||
order_clause = order_clause.empty() ? op_str : order_clause + "," + op_str;
|
||||
};
|
||||
|
||||
set_main_query_clause(last_op, last_pos);
|
||||
|
||||
operation_pos.pop_back();
|
||||
|
||||
if (kql_parser[last_op].need_input)
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
while (!operation_pos.empty())
|
||||
{
|
||||
auto prev_op = operation_pos.back().first;
|
||||
auto prev_pos = operation_pos.back().second;
|
||||
|
||||
if (kql_parser[prev_op].gen_output)
|
||||
break;
|
||||
if (!project_clause.empty() && prev_op == "project")
|
||||
break;
|
||||
set_main_query_clause(prev_op, prev_pos);
|
||||
operation_pos.pop_back();
|
||||
last_op = prev_op;
|
||||
last_pos = prev_pos;
|
||||
}
|
||||
}
|
||||
|
||||
if (!operation_pos.empty())
|
||||
{
|
||||
for (auto i = 0; i< kql_parser[last_op].backspace_steps; ++i)
|
||||
--last_pos;
|
||||
|
||||
String sub_query = std::format("({})", String(operation_pos.front().second->begin, last_pos->end));
|
||||
Tokens token_subquery(sub_query.c_str(), sub_query.c_str() + sub_query.size());
|
||||
IParser::Pos pos_subquery(token_subquery, pos.max_depth);
|
||||
|
||||
if (!ParserKQLSubquery().parse(pos_subquery, tables, expected))
|
||||
return false;
|
||||
select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!ParserKQLTable().parse(last_pos, node, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
auto set_query_clasue =[&](String op_str, String op_calsue)
|
||||
{
|
||||
auto oprator = getOperator(op_str);
|
||||
if (oprator)
|
||||
{
|
||||
Tokens token_clause(op_calsue.c_str(), op_calsue.c_str() + op_calsue.size());
|
||||
IParser::Pos pos_clause(token_clause, pos.max_depth);
|
||||
if (!oprator->parse(pos_clause, node, expected))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!select_query->select())
|
||||
{
|
||||
if (project_clause.empty())
|
||||
project_clause = "*";
|
||||
if (!set_query_clasue("project", project_clause))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!order_clause.empty())
|
||||
if (!set_query_clasue("order by", order_clause))
|
||||
return false;
|
||||
|
||||
if (!where_clause.empty())
|
||||
if (!set_query_clasue("where", where_clause))
|
||||
return false;
|
||||
|
||||
if (!limit_clause.empty())
|
||||
if (!set_query_clasue("limit", limit_clause))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (!select_query->select())
|
||||
{
|
||||
auto expr = String("*");
|
||||
Tokens tokens(expr.c_str(), expr.c_str()+expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
if (!std::make_unique<ParserKQLProject>()->parse(new_pos, node, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_node;
|
||||
|
||||
if (!ParserKQLTaleFunction().parse(pos, select_node, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr node_subquery = std::make_shared<ASTSubquery>();
|
||||
node_subquery->children.push_back(select_node);
|
||||
|
||||
ASTPtr node_table_expr = std::make_shared<ASTTableExpression>();
|
||||
node_table_expr->as<ASTTableExpression>()->subquery = node_subquery;
|
||||
|
||||
node_table_expr->children.emplace_back(node_subquery);
|
||||
|
||||
ASTPtr node_table_in_select_query_emlement = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
node_table_in_select_query_emlement->as<ASTTablesInSelectQueryElement>()->table_expression = node_table_expr;
|
||||
|
||||
ASTPtr res = std::make_shared<ASTTablesInSelectQuery>();
|
||||
|
||||
res->children.emplace_back(node_table_in_select_query_emlement);
|
||||
|
||||
node = res;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
32
src/Parsers/Kusto/ParserKQLQuery.h
Normal file
32
src/Parsers/Kusto/ParserKQLQuery.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ParserKQLBase : public IParserBase
|
||||
{
|
||||
public:
|
||||
static String getExprFromToken(Pos & pos);
|
||||
static String getExprFromPipe(Pos & pos);
|
||||
static String getExprFromToken(const String & text, const uint32_t & max_depth);
|
||||
};
|
||||
|
||||
class ParserKQLQuery : public IParserBase
|
||||
{
|
||||
|
||||
protected:
|
||||
static std::unique_ptr<IParserBase> getOperator(String &op_name);
|
||||
const char * getName() const override { return "KQL query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserKQLSubquery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL subquery"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
60
src/Parsers/Kusto/ParserKQLSort.cpp
Normal file
60
src/Parsers/Kusto/ParserKQLSort.cpp
Normal file
@ -0,0 +1,60 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLSort.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool has_dir = false;
|
||||
std::vector <bool> has_directions;
|
||||
ParserOrderByExpressionList order_list;
|
||||
ASTPtr order_expression_list;
|
||||
|
||||
auto expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
auto pos_backup = new_pos;
|
||||
if (!order_list.parse(pos_backup, order_expression_list, expected))
|
||||
return false;
|
||||
|
||||
while (!new_pos->isEnd() && new_pos->type != TokenType::PipeMark && new_pos->type != TokenType::Semicolon)
|
||||
{
|
||||
String tmp(new_pos->begin, new_pos->end);
|
||||
if (tmp == "desc" or tmp == "asc")
|
||||
has_dir = true;
|
||||
|
||||
if (new_pos->type == TokenType::Comma)
|
||||
{
|
||||
has_directions.push_back(has_dir);
|
||||
has_dir = false;
|
||||
}
|
||||
++new_pos;
|
||||
}
|
||||
has_directions.push_back(has_dir);
|
||||
|
||||
for (uint64_t i = 0; i < order_expression_list->children.size(); ++i)
|
||||
{
|
||||
if (!has_directions[i])
|
||||
{
|
||||
auto *order_expr = order_expression_list->children[i]->as<ASTOrderByElement>();
|
||||
order_expr->direction = -1; // default desc
|
||||
if (!order_expr->nulls_direction_was_explicitly_specified)
|
||||
order_expr->nulls_direction = -1;
|
||||
else
|
||||
order_expr->nulls_direction = order_expr->nulls_direction == 1 ? -1 : 1;
|
||||
}
|
||||
}
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(order_expression_list));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLSort.h
Normal file
16
src/Parsers/Kusto/ParserKQLSort.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLSort : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL order by"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
102
src/Parsers/Kusto/ParserKQLStatement.cpp
Normal file
102
src/Parsers/Kusto/ParserKQLStatement.cpp
Normal file
@ -0,0 +1,102 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLStatement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKQLWithOutput query_with_output_p(end, allow_settings_after_format_in_insert);
|
||||
ParserSetQuery set_p;
|
||||
|
||||
bool res = query_with_output_p.parse(pos, node, expected)
|
||||
|| set_p.parse(pos, node, expected);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
bool ParserKQLWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKQLWithUnionQuery kql_p;
|
||||
|
||||
ASTPtr query;
|
||||
bool parsed = kql_p.parse(pos, query, expected);
|
||||
|
||||
if (!parsed)
|
||||
return false;
|
||||
|
||||
node = std::move(query);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr kql_query;
|
||||
|
||||
if (!ParserKQLQuery().parse(pos, kql_query, expected))
|
||||
return false;
|
||||
|
||||
if (kql_query->as<ASTSelectWithUnionQuery>())
|
||||
{
|
||||
node = std::move(kql_query);
|
||||
return true;
|
||||
}
|
||||
|
||||
auto list_node = std::make_shared<ASTExpressionList>();
|
||||
list_node->children.push_back(kql_query);
|
||||
|
||||
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
|
||||
node = select_with_union_query;
|
||||
select_with_union_query->list_of_selects = list_node;
|
||||
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLTaleFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKQLWithUnionQuery kql_p;
|
||||
ASTPtr select;
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
|
||||
auto begin = pos;
|
||||
auto paren_count = 0 ;
|
||||
String kql_statement;
|
||||
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
++paren_count;
|
||||
while (!pos->isEnd())
|
||||
{
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--paren_count;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++paren_count;
|
||||
|
||||
if (paren_count == 0)
|
||||
break;
|
||||
|
||||
kql_statement = kql_statement + " " + String(pos->begin,pos->end);
|
||||
++pos;
|
||||
}
|
||||
|
||||
Tokens token_kql(kql_statement.c_str(), kql_statement.c_str() + kql_statement.size());
|
||||
IParser::Pos pos_kql(token_kql, pos.max_depth);
|
||||
|
||||
if (kql_p.parse(pos_kql, select, expected))
|
||||
{
|
||||
node = select;
|
||||
++pos;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
pos = begin;
|
||||
return false;
|
||||
};
|
||||
|
||||
}
|
52
src/Parsers/Kusto/ParserKQLStatement.h
Normal file
52
src/Parsers/Kusto/ParserKQLStatement.h
Normal file
@ -0,0 +1,52 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLStatement : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
const char * getName() const override { return "KQL Statement"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserKQLStatement(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
class ParserKQLWithOutput : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
const char * getName() const override { return "KQL with output"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserKQLWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
class ParserKQLWithUnionQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL query, possibly with UNION"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserKQLTaleFunction : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL() function"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
81
src/Parsers/Kusto/ParserKQLSummarize.cpp
Normal file
81
src/Parsers/Kusto/ParserKQLSummarize.cpp
Normal file
@ -0,0 +1,81 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInterpolateElement.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLSummarize.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <Parsers/ParserWithElement.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
ASTPtr group_expression_list;
|
||||
|
||||
String expr_aggregation;
|
||||
String expr_groupby;
|
||||
String expr_columns;
|
||||
bool groupby = false;
|
||||
|
||||
auto begin = pos;
|
||||
auto pos_groupby = pos;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin, pos->end) == "by")
|
||||
{
|
||||
groupby = true;
|
||||
auto end = pos;
|
||||
--end;
|
||||
expr_aggregation = begin <= end ? String(begin->begin, end->end) : "";
|
||||
pos_groupby = pos;
|
||||
++pos_groupby;
|
||||
}
|
||||
++pos;
|
||||
}
|
||||
--pos;
|
||||
if (groupby)
|
||||
expr_groupby = String(pos_groupby->begin, pos->end);
|
||||
else
|
||||
expr_aggregation = begin <= pos ? String(begin->begin, pos->end) : "";
|
||||
|
||||
auto expr_aggregation_str = expr_aggregation.empty() ? "" : expr_aggregation +",";
|
||||
expr_columns = groupby ? expr_aggregation_str + expr_groupby : expr_aggregation_str;
|
||||
|
||||
String converted_columns = getExprFromToken(expr_columns, pos.max_depth);
|
||||
|
||||
Tokens token_converted_columns(converted_columns.c_str(), converted_columns.c_str() + converted_columns.size());
|
||||
IParser::Pos pos_converted_columns(token_converted_columns, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(true).parse(pos_converted_columns, select_expression_list, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
|
||||
if (groupby)
|
||||
{
|
||||
String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth);
|
||||
|
||||
Tokens token_converted_groupby(converted_groupby.c_str(), converted_groupby.c_str() + converted_groupby.size());
|
||||
IParser::Pos postoken_converted_groupby(token_converted_groupby, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(false).parse(postoken_converted_groupby, group_expression_list, expected))
|
||||
return false;
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::move(group_expression_list));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
17
src/Parsers/Kusto/ParserKQLSummarize.h
Normal file
17
src/Parsers/Kusto/ParserKQLSummarize.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLSummarize : public ParserKQLBase
|
||||
{
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "KQL summarize"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
55
src/Parsers/Kusto/ParserKQLTable.cpp
Normal file
55
src/Parsers/Kusto/ParserKQLTable.cpp
Normal file
@ -0,0 +1,55 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLTable.h>
|
||||
#include <unordered_set>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLTable :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
std::unordered_set<String> sql_keywords
|
||||
({
|
||||
"SELECT",
|
||||
"INSERT",
|
||||
"CREATE",
|
||||
"ALTER",
|
||||
"SYSTEM",
|
||||
"SHOW",
|
||||
"GRANT",
|
||||
"REVOKE",
|
||||
"ATTACH",
|
||||
"CHECK",
|
||||
"DESCRIBE",
|
||||
"DESC",
|
||||
"DETACH",
|
||||
"DROP",
|
||||
"EXISTS",
|
||||
"KILL",
|
||||
"OPTIMIZE",
|
||||
"RENAME",
|
||||
"SET",
|
||||
"TRUNCATE",
|
||||
"USE",
|
||||
"EXPLAIN"
|
||||
});
|
||||
|
||||
ASTPtr tables;
|
||||
String table_name(pos->begin,pos->end);
|
||||
String table_name_upcase(table_name);
|
||||
|
||||
std::transform(table_name_upcase.begin(), table_name_upcase.end(),table_name_upcase.begin(), toupper);
|
||||
|
||||
if (sql_keywords.find(table_name_upcase) != sql_keywords.end())
|
||||
return false;
|
||||
|
||||
if (!ParserTablesInSelectQuery().parse(pos, tables, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
17
src/Parsers/Kusto/ParserKQLTable.h
Normal file
17
src/Parsers/Kusto/ParserKQLTable.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLTable : public ParserKQLBase
|
||||
{
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "KQL Table"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -338,7 +338,7 @@ Token Lexer::nextTokenImpl()
|
||||
++pos;
|
||||
if (pos < end && *pos == '|')
|
||||
return Token(TokenType::Concatenation, token_begin, ++pos);
|
||||
return Token(TokenType::ErrorSinglePipeMark, token_begin, pos);
|
||||
return Token(TokenType::PipeMark, token_begin, pos);
|
||||
}
|
||||
case '@':
|
||||
{
|
||||
|
@ -51,6 +51,7 @@ namespace DB
|
||||
M(Greater) \
|
||||
M(LessOrEquals) \
|
||||
M(GreaterOrEquals) \
|
||||
M(PipeMark) \
|
||||
M(Concatenation) /** String concatenation operator: || */ \
|
||||
\
|
||||
M(At) /** @. Used for specifying user names and also for MySQL-style variables. */ \
|
||||
|
@ -20,7 +20,7 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
|
||||
ParserKeyword s_on("ON");
|
||||
ParserIdentifier function_name_p;
|
||||
ParserKeyword s_as("AS");
|
||||
ParserLambdaExpression lambda_p;
|
||||
ParserExpression lambda_p;
|
||||
|
||||
ASTPtr function_name;
|
||||
ASTPtr function_core;
|
||||
|
@ -152,7 +152,7 @@ bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserKeyword s_assume("ASSUME");
|
||||
|
||||
ParserIdentifier name_p;
|
||||
ParserLogicalOrExpression expression_p;
|
||||
ParserExpression expression_p;
|
||||
|
||||
ASTPtr name;
|
||||
ASTPtr expr;
|
||||
@ -858,8 +858,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserStorage storage_p;
|
||||
ParserStorage storage_inner;
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserIntervalOperatorExpression watermark_p;
|
||||
ParserIntervalOperatorExpression lateness_p;
|
||||
ParserExpression watermark_p;
|
||||
ParserExpression lateness_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
|
||||
ASTPtr table;
|
||||
|
@ -134,7 +134,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserKeyword s_remove{"REMOVE"};
|
||||
ParserKeyword s_type{"TYPE"};
|
||||
ParserKeyword s_collate{"COLLATE"};
|
||||
ParserTernaryOperatorExpression expr_parser;
|
||||
ParserExpression expr_parser;
|
||||
ParserStringLiteral string_literal_parser;
|
||||
ParserLiteral literal_parser;
|
||||
ParserCodec codec_parser;
|
||||
|
@ -20,7 +20,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
|
||||
ParserLiteral default_parser;
|
||||
ParserArrayOfLiterals array_literals_parser;
|
||||
ParserTernaryOperatorExpression expression_parser;
|
||||
ParserExpression expression_parser;
|
||||
|
||||
/// mandatory attribute name
|
||||
ASTPtr name;
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Parsers/ASTExternalDDLQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserDropQuery.h>
|
||||
#include <Parsers/ParserExternalDDLQuery.h>
|
||||
#include <Parsers/ParserRenameQuery.h>
|
||||
|
@ -226,7 +226,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
|
||||
else if (ParserKeyword("ON").ignore(pos, expected))
|
||||
{
|
||||
/// OR is operator with lowest priority, so start parsing from it.
|
||||
if (!ParserLogicalOrExpression().parse(pos, table_join->on_expression, expected))
|
||||
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Parsers/ParserAttachAccessEntity.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <string_view>
|
||||
#include <regex>
|
||||
#include <gtest/gtest.h>
|
||||
@ -292,3 +293,185 @@ INSTANTIATE_TEST_SUITE_P(ParserAttachUserQuery, ParserTest,
|
||||
"^$"
|
||||
}
|
||||
})));
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(std::make_shared<ParserKQLQuery>()),
|
||||
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
|
||||
{
|
||||
"Customers",
|
||||
"SELECT *\nFROM Customers"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation",
|
||||
"SELECT\n FirstName,\n LastName,\n Occupation\nFROM Customers"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | take 3",
|
||||
"SELECT\n FirstName,\n LastName,\n Occupation\nFROM Customers\nLIMIT 3"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | limit 3",
|
||||
"SELECT\n FirstName,\n LastName,\n Occupation\nFROM Customers\nLIMIT 3"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | take 1 | take 3",
|
||||
"SELECT *\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 1\n)\nLIMIT 3"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | take 3 | take 1",
|
||||
"SELECT *\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 3\n)\nLIMIT 1"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName",
|
||||
"SELECT\n FirstName,\n LastName\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 3\n)"
|
||||
},
|
||||
{
|
||||
"Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education",
|
||||
"SELECT\n FirstName,\n LastName,\n Education\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 3\n)"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName desc",
|
||||
"SELECT *\nFROM Customers\nORDER BY FirstName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | take 3 | order by FirstName desc",
|
||||
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n LIMIT 3\n)\nORDER BY FirstName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName asc",
|
||||
"SELECT *\nFROM Customers\nORDER BY FirstName ASC"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName",
|
||||
"SELECT *\nFROM Customers\nORDER BY FirstName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | order by LastName",
|
||||
"SELECT *\nFROM Customers\nORDER BY LastName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | order by Age desc , FirstName asc ",
|
||||
"SELECT *\nFROM Customers\nORDER BY\n Age DESC,\n FirstName ASC"
|
||||
},
|
||||
{
|
||||
"Customers | order by Age asc , FirstName desc",
|
||||
"SELECT *\nFROM Customers\nORDER BY\n Age ASC,\n FirstName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName | order by Age ",
|
||||
"SELECT *\nFROM Customers\nORDER BY\n Age DESC,\n FirstName DESC"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName nulls first",
|
||||
"SELECT *\nFROM Customers\nORDER BY FirstName DESC NULLS FIRST"
|
||||
},
|
||||
{
|
||||
"Customers | sort by FirstName nulls last",
|
||||
"SELECT *\nFROM Customers\nORDER BY FirstName DESC NULLS LAST"
|
||||
},
|
||||
{
|
||||
"Customers | where Occupation == 'Skilled Manual'",
|
||||
"SELECT *\nFROM Customers\nWHERE Occupation = 'Skilled Manual'"
|
||||
},
|
||||
{
|
||||
"Customers | where Occupation != 'Skilled Manual'",
|
||||
"SELECT *\nFROM Customers\nWHERE Occupation != 'Skilled Manual'"
|
||||
},
|
||||
{
|
||||
"Customers |where Education in ('Bachelors','High School')",
|
||||
"SELECT *\nFROM Customers\nWHERE Education IN ('Bachelors', 'High School')"
|
||||
},
|
||||
{
|
||||
"Customers | where Education !in ('Bachelors','High School')",
|
||||
"SELECT *\nFROM Customers\nWHERE Education NOT IN ('Bachelors', 'High School')"
|
||||
},
|
||||
{
|
||||
"Customers |where Education contains_cs 'Degree'",
|
||||
"SELECT *\nFROM Customers\nWHERE Education LIKE '%Degree%'"
|
||||
},
|
||||
{
|
||||
"Customers | where Occupation startswith_cs 'Skil'",
|
||||
"SELECT *\nFROM Customers\nWHERE startsWith(Occupation, 'Skil')"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName endswith_cs 'le'",
|
||||
"SELECT *\nFROM Customers\nWHERE endsWith(FirstName, 'le')"
|
||||
},
|
||||
{
|
||||
"Customers | where Age == 26",
|
||||
"SELECT *\nFROM Customers\nWHERE Age = 26"
|
||||
},
|
||||
{
|
||||
"Customers | where Age > 20 and Age < 30",
|
||||
"SELECT *\nFROM Customers\nWHERE (Age > 20) AND (Age < 30)"
|
||||
},
|
||||
{
|
||||
"Customers | where Age > 30 | where Education == 'Bachelors'",
|
||||
"SELECT *\nFROM Customers\nWHERE (Education = 'Bachelors') AND (Age > 30)"
|
||||
},
|
||||
{
|
||||
"Customers |summarize count() by Occupation",
|
||||
"SELECT\n count(),\n Occupation\nFROM Customers\nGROUP BY Occupation"
|
||||
},
|
||||
{
|
||||
"Customers|summarize sum(Age) by Occupation",
|
||||
"SELECT\n sum(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation"
|
||||
},
|
||||
{
|
||||
"Customers|summarize avg(Age) by Occupation",
|
||||
"SELECT\n avg(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation"
|
||||
},
|
||||
{
|
||||
"Customers|summarize min(Age) by Occupation",
|
||||
"SELECT\n min(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation"
|
||||
},
|
||||
{
|
||||
"Customers |summarize max(Age) by Occupation",
|
||||
"SELECT\n max(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName contains 'pet'",
|
||||
"SELECT *\nFROM Customers\nWHERE FirstName ILIKE '%pet%'"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName !contains 'pet'",
|
||||
"SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE '%pet%')"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName endswith 'er'",
|
||||
"SELECT *\nFROM Customers\nWHERE FirstName ILIKE '%er'"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName !endswith 'er'",
|
||||
"SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE '%er')"
|
||||
},
|
||||
{
|
||||
"Customers | where Education has 'School'",
|
||||
"SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Education, 'School')"
|
||||
},
|
||||
{
|
||||
"Customers | where Education !has 'School'",
|
||||
"SELECT *\nFROM Customers\nWHERE NOT hasTokenCaseInsensitive(Education, 'School')"
|
||||
},
|
||||
{
|
||||
"Customers | where Education has_cs 'School'",
|
||||
"SELECT *\nFROM Customers\nWHERE hasToken(Education, 'School')"
|
||||
},
|
||||
{
|
||||
"Customers | where Education !has_cs 'School'",
|
||||
"SELECT *\nFROM Customers\nWHERE NOT hasToken(Education, 'School')"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName matches regex 'P.*r'",
|
||||
"SELECT *\nFROM Customers\nWHERE match(FirstName, 'P.*r')"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName startswith 'pet'",
|
||||
"SELECT *\nFROM Customers\nWHERE FirstName ILIKE 'pet%'"
|
||||
},
|
||||
{
|
||||
"Customers | where FirstName !startswith 'pet'",
|
||||
"SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE 'pet%')"
|
||||
}
|
||||
})));
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/LightweightDeleteDescription.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
@ -1056,6 +1057,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add",
|
||||
ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
|
||||
throw Exception{"Cannot add column " + backQuote(column_name) + ": this column name is reserved for lightweight delete feature",
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
if (command.codec)
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
|
||||
|
||||
@ -1240,6 +1245,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": column with this name already exists",
|
||||
ErrorCodes::DUPLICATE_COLUMN};
|
||||
|
||||
if (command.rename_to == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
|
||||
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": this column name is reserved for lightweight delete feature",
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
if (modified_columns.contains(column_name))
|
||||
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query",
|
||||
ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
@ -35,7 +35,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS(
|
||||
AsynchronousReaderPtr reader_, const ReadSettings & settings_, std::shared_ptr<ReadBufferFromHDFS> impl_)
|
||||
IAsynchronousReader & reader_, const ReadSettings & settings_, std::shared_ptr<ReadBufferFromHDFS> impl_)
|
||||
: BufferWithOwnMemory<SeekableReadBuffer>(settings_.remote_fs_buffer_size)
|
||||
, reader(reader_)
|
||||
, priority(settings_.priority)
|
||||
@ -72,7 +72,7 @@ std::future<IAsynchronousReader::Result> AsynchronousReadBufferFromHDFS::asyncRe
|
||||
request.offset = file_offset_of_buffer_end;
|
||||
request.priority = priority;
|
||||
request.ignore = 0;
|
||||
return reader->submit(request);
|
||||
return reader.submit(request);
|
||||
}
|
||||
|
||||
void AsynchronousReadBufferFromHDFS::prefetch()
|
||||
|
@ -24,7 +24,7 @@ class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBu
|
||||
{
|
||||
public:
|
||||
AsynchronousReadBufferFromHDFS(
|
||||
AsynchronousReaderPtr reader_,
|
||||
IAsynchronousReader & reader_,
|
||||
const ReadSettings & settings_,
|
||||
std::shared_ptr<ReadBufferFromHDFS> impl_);
|
||||
|
||||
@ -51,7 +51,7 @@ private:
|
||||
|
||||
std::future<IAsynchronousReader::Result> asyncReadInto(char * data, size_t size);
|
||||
|
||||
AsynchronousReaderPtr reader;
|
||||
IAsynchronousReader & reader;
|
||||
Int32 priority;
|
||||
std::shared_ptr<ReadBufferFromHDFS> impl;
|
||||
std::future<IAsynchronousReader::Result> prefetch_future;
|
||||
|
@ -7451,8 +7451,9 @@ String StorageReplicatedMergeTree::getTableSharedID() const
|
||||
/// can be called only during table initialization
|
||||
std::lock_guard lock(table_shared_id_mutex);
|
||||
|
||||
bool maybe_has_metadata_in_zookeeper = !has_metadata_in_zookeeper.has_value() || *has_metadata_in_zookeeper;
|
||||
/// Can happen if table was partially initialized before drop by DatabaseCatalog
|
||||
if (table_shared_id == UUIDHelpers::Nil)
|
||||
if (maybe_has_metadata_in_zookeeper && table_shared_id == UUIDHelpers::Nil)
|
||||
createTableSharedID();
|
||||
|
||||
return toString(table_shared_id);
|
||||
|
@ -508,7 +508,7 @@ std::unique_ptr<ReadBuffer> StorageS3Source::createS3ReadBuffer(const String & k
|
||||
LOG_TRACE(
|
||||
log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size);
|
||||
|
||||
return std::make_unique<ParallelReadBuffer>(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num);
|
||||
return std::make_unique<ParallelReadBuffer>(std::move(factory), threadPoolCallbackRunner<void>(IOThreadPool::get(), "S3ParallelRead"), download_thread_num);
|
||||
}
|
||||
|
||||
String StorageS3Source::getName() const
|
||||
@ -617,7 +617,7 @@ public:
|
||||
s3_configuration_.rw_settings,
|
||||
std::nullopt,
|
||||
DBMS_DEFAULT_BUFFER_SIZE,
|
||||
threadPoolCallbackRunner(IOThreadPool::get()),
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "S3ParallelRead"),
|
||||
context->getWriteSettings()),
|
||||
compression_method,
|
||||
3);
|
||||
|
@ -351,7 +351,7 @@ namespace
|
||||
return wrapReadBufferWithCompressionMethod(
|
||||
std::make_unique<ParallelReadBuffer>(
|
||||
std::move(read_buffer_factory),
|
||||
threadPoolCallbackRunner(IOThreadPool::get()),
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "URLParallelRead"),
|
||||
download_threads),
|
||||
compression_method,
|
||||
settings.zstd_window_log_max);
|
||||
|
@ -25,7 +25,7 @@ int main()
|
||||
String path = "/path/to/hdfs/file";
|
||||
ReadSettings settings = {};
|
||||
auto in = std::make_unique<ReadBufferFromHDFS>(hdfs_namenode_url, path, *config, settings);
|
||||
auto reader = IObjectStorage::getThreadPoolReader();
|
||||
auto & reader = IObjectStorage::getThreadPoolReader();
|
||||
AsynchronousReadBufferFromHDFS buf(reader, {}, std::move(in));
|
||||
|
||||
String output;
|
||||
|
@ -61,7 +61,7 @@ namespace DB
|
||||
ColumnsDescription /*cached_columns_*/) const
|
||||
{
|
||||
const Settings & settings = context_->getSettings();
|
||||
ParserLambdaExpression partition_by_parser;
|
||||
ParserExpression partition_by_parser;
|
||||
ASTPtr partition_by_ast = parseQuery(
|
||||
partition_by_parser,
|
||||
"(" + partition_by_def + ")",
|
||||
|
@ -136,6 +136,7 @@ CI_CONFIG = {
|
||||
"build_type": "",
|
||||
"sanitizer": "",
|
||||
"package_type": "binary",
|
||||
"static_binary_name": "aarch64v80compat",
|
||||
"libraries": "static",
|
||||
"tidy": "disable",
|
||||
"with_coverage": False,
|
||||
|
@ -3,7 +3,7 @@ set -xeuo pipefail
|
||||
|
||||
echo "Running prepare script"
|
||||
export DEBIAN_FRONTEND=noninteractive
|
||||
export RUNNER_VERSION=2.293.0
|
||||
export RUNNER_VERSION=2.296.2
|
||||
export RUNNER_HOME=/home/ubuntu/actions-runner
|
||||
|
||||
deb_arch() {
|
||||
|
@ -865,6 +865,12 @@ class TestCase:
|
||||
stdout=PIPE,
|
||||
universal_newlines=True,
|
||||
).communicate()[0]
|
||||
if diff.startswith("Binary files "):
|
||||
diff += "Content of stdout:\n===================\n"
|
||||
file = open(self.stdout_file, "r")
|
||||
diff += str(file.read())
|
||||
file.close()
|
||||
diff += "==================="
|
||||
description += f"\n{diff}\n"
|
||||
if debug_log:
|
||||
description += "\n"
|
||||
|
41
tests/integration/helpers/keeper_utils.py
Normal file
41
tests/integration/helpers/keeper_utils.py
Normal file
@ -0,0 +1,41 @@
|
||||
import socket
|
||||
import time
|
||||
|
||||
|
||||
def get_keeper_socket(cluster, node, port=9181):
|
||||
hosts = cluster.get_instance_ip(node.name)
|
||||
client = socket.socket()
|
||||
client.settimeout(10)
|
||||
client.connect((hosts, port))
|
||||
return client
|
||||
|
||||
|
||||
def send_4lw_cmd(cluster, node, cmd="ruok", port=9181):
|
||||
client = None
|
||||
try:
|
||||
client = get_keeper_socket(cluster, node, port)
|
||||
client.send(cmd.encode())
|
||||
data = client.recv(100_000)
|
||||
data = data.decode()
|
||||
return data
|
||||
finally:
|
||||
if client is not None:
|
||||
client.close()
|
||||
|
||||
|
||||
NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests"
|
||||
|
||||
|
||||
def wait_until_connected(cluster, node, port=9181):
|
||||
while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG:
|
||||
time.sleep(0.1)
|
||||
|
||||
|
||||
def wait_until_quorum_lost(cluster, node, port=9181):
|
||||
while send_4lw_cmd(cluster, node, "mntr", port) != NOT_SERVING_REQUESTS_ERROR_MSG:
|
||||
time.sleep(0.1)
|
||||
|
||||
|
||||
def wait_nodes(cluster, nodes):
|
||||
for node in nodes:
|
||||
wait_until_connected(cluster, node)
|
@ -15,6 +15,7 @@ node1 = cluster.add_instance(
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
@ -1,22 +0,0 @@
|
||||
<clickhouse>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>1</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>10000</session_timeout_ms>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>9234</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</clickhouse>
|
@ -1,8 +0,0 @@
|
||||
<clickhouse>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</clickhouse>
|
@ -2,6 +2,7 @@ import os
|
||||
import pytest
|
||||
import socket
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
import helpers.keeper_utils as keeper_utils
|
||||
import time
|
||||
|
||||
|
||||
@ -62,37 +63,6 @@ def get_fake_zk(nodename, timeout=30.0):
|
||||
return _fake_zk_instance
|
||||
|
||||
|
||||
def get_keeper_socket(node_name):
|
||||
hosts = cluster.get_instance_ip(node_name)
|
||||
client = socket.socket()
|
||||
client.settimeout(10)
|
||||
client.connect((hosts, 9181))
|
||||
return client
|
||||
|
||||
|
||||
def send_4lw_cmd(node_name, cmd="ruok"):
|
||||
client = None
|
||||
try:
|
||||
client = get_keeper_socket(node_name)
|
||||
client.send(cmd.encode())
|
||||
data = client.recv(100_000)
|
||||
data = data.decode()
|
||||
return data
|
||||
finally:
|
||||
if client is not None:
|
||||
client.close()
|
||||
|
||||
|
||||
def wait_until_connected(node_name):
|
||||
while send_4lw_cmd(node_name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG:
|
||||
time.sleep(0.1)
|
||||
|
||||
|
||||
def wait_nodes(nodes):
|
||||
for node in nodes:
|
||||
wait_until_connected(node.name)
|
||||
|
||||
|
||||
def wait_and_assert_data(zk, path, data):
|
||||
while zk.retry(zk.exists, path) is None:
|
||||
time.sleep(0.1)
|
||||
@ -104,9 +74,6 @@ def close_zk(zk):
|
||||
zk.close()
|
||||
|
||||
|
||||
NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests"
|
||||
|
||||
|
||||
def test_cluster_recovery(started_cluster):
|
||||
node_zks = []
|
||||
try:
|
||||
@ -114,7 +81,7 @@ def test_cluster_recovery(started_cluster):
|
||||
for node in nodes[CLUSTER_SIZE:]:
|
||||
node.stop_clickhouse()
|
||||
|
||||
wait_nodes(nodes[:CLUSTER_SIZE])
|
||||
keeper_utils.wait_nodes(cluster, nodes[:CLUSTER_SIZE])
|
||||
|
||||
node_zks = [get_fake_zk(node.name) for node in nodes[:CLUSTER_SIZE]]
|
||||
|
||||
@ -152,7 +119,7 @@ def test_cluster_recovery(started_cluster):
|
||||
wait_and_assert_data(node_zk, "/test_force_recovery_extra", "somedataextra")
|
||||
|
||||
nodes[0].start_clickhouse()
|
||||
wait_until_connected(nodes[0].name)
|
||||
keeper_utils.wait_until_connected(cluster, nodes[0])
|
||||
node_zks[0] = get_fake_zk(nodes[0].name)
|
||||
wait_and_assert_data(node_zks[0], "/test_force_recovery_extra", "somedataextra")
|
||||
|
||||
@ -167,8 +134,7 @@ def test_cluster_recovery(started_cluster):
|
||||
node.stop_clickhouse()
|
||||
|
||||
# wait for node1 to lose quorum
|
||||
while send_4lw_cmd(nodes[0].name, "mntr") != NOT_SERVING_REQUESTS_ERROR_MSG:
|
||||
time.sleep(0.2)
|
||||
keeper_utils.wait_until_quorum_lost(cluster, nodes[0])
|
||||
|
||||
nodes[0].copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "recovered_keeper1.xml"),
|
||||
@ -177,9 +143,15 @@ def test_cluster_recovery(started_cluster):
|
||||
|
||||
nodes[0].query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
assert send_4lw_cmd(nodes[0].name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG
|
||||
send_4lw_cmd(nodes[0].name, "rcvr")
|
||||
assert send_4lw_cmd(nodes[0].name, "mntr") == NOT_SERVING_REQUESTS_ERROR_MSG
|
||||
assert (
|
||||
keeper_utils.send_4lw_cmd(cluster, nodes[0], "mntr")
|
||||
== keeper_utils.NOT_SERVING_REQUESTS_ERROR_MSG
|
||||
)
|
||||
keeper_utils.send_4lw_cmd(cluster, nodes[0], "rcvr")
|
||||
assert (
|
||||
keeper_utils.send_4lw_cmd(cluster, nodes[0], "mntr")
|
||||
== keeper_utils.NOT_SERVING_REQUESTS_ERROR_MSG
|
||||
)
|
||||
|
||||
# add one node to restore the quorum
|
||||
nodes[CLUSTER_SIZE].copy_file_to_container(
|
||||
@ -191,10 +163,10 @@ def test_cluster_recovery(started_cluster):
|
||||
)
|
||||
|
||||
nodes[CLUSTER_SIZE].start_clickhouse()
|
||||
wait_until_connected(nodes[CLUSTER_SIZE].name)
|
||||
keeper_utils.wait_until_connected(cluster, nodes[CLUSTER_SIZE])
|
||||
|
||||
# node1 should have quorum now and accept requests
|
||||
wait_until_connected(nodes[0].name)
|
||||
keeper_utils.wait_until_connected(cluster, nodes[0])
|
||||
|
||||
node_zks.append(get_fake_zk(nodes[CLUSTER_SIZE].name))
|
||||
|
||||
@ -206,7 +178,7 @@ def test_cluster_recovery(started_cluster):
|
||||
f"/etc/clickhouse-server/config.d/enable_keeper{i+1}.xml",
|
||||
)
|
||||
node.start_clickhouse()
|
||||
wait_until_connected(node.name)
|
||||
keeper_utils.wait_until_connected(cluster, node)
|
||||
node_zks.append(get_fake_zk(node.name))
|
||||
|
||||
# refresh old zk sessions
|
||||
@ -223,7 +195,7 @@ def test_cluster_recovery(started_cluster):
|
||||
wait_and_assert_data(node_zks[-1], "/test_force_recovery_last", "somedatalast")
|
||||
|
||||
nodes[0].start_clickhouse()
|
||||
wait_until_connected(nodes[0].name)
|
||||
keeper_utils.wait_until_connected(cluster, nodes[0])
|
||||
node_zks[0] = get_fake_zk(nodes[0].name)
|
||||
for zk in node_zks[:nodes_left]:
|
||||
assert_all_data(zk)
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user