Merge branch 'master' into add-dmesg-log-to-fuzzer

This commit is contained in:
Alexey Milovidov 2023-01-13 05:22:18 +03:00 committed by GitHub
commit dff5e3eafd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
153 changed files with 3791 additions and 942 deletions

View File

@ -144,6 +144,13 @@
# define TSA_REQUIRES_SHARED(...) __attribute__((requires_shared_capability(__VA_ARGS__))) /// thread needs shared possession of given capability
# define TSA_ACQUIRED_AFTER(...) __attribute__((acquired_after(__VA_ARGS__))) /// annotated lock must be locked after given lock
# define TSA_NO_THREAD_SAFETY_ANALYSIS __attribute__((no_thread_safety_analysis)) /// disable TSA for a function
# define TSA_CAPABILITY(...) __attribute__((capability(__VA_ARGS__))) /// object of a class can be used as capability
# define TSA_ACQUIRE(...) __attribute__((acquire_capability(__VA_ARGS__))) /// function acquires a capability, but does not release it
# define TSA_TRY_ACQUIRE(...) __attribute__((try_acquire_capability(__VA_ARGS__))) /// function tries to acquire a capability and returns a boolean value indicating success or failure
# define TSA_RELEASE(...) __attribute__((release_capability(__VA_ARGS__))) /// function releases the given capability
# define TSA_ACQUIRE_SHARED(...) __attribute__((acquire_shared_capability(__VA_ARGS__))) /// function acquires a shared capability, but does not release it
# define TSA_TRY_ACQUIRE_SHARED(...) __attribute__((try_acquire_shared_capability(__VA_ARGS__))) /// function tries to acquire a shared capability and returns a boolean value indicating success or failure
# define TSA_RELEASE_SHARED(...) __attribute__((release_shared_capability(__VA_ARGS__))) /// function releases the given shared capability
/// Macros for suppressing TSA warnings for specific reads/writes (instead of suppressing it for the whole function)
/// They use a lambda function to apply function attribute to a single statement. This enable us to suppress warnings locally instead of
@ -164,6 +171,13 @@
# define TSA_REQUIRES(...)
# define TSA_REQUIRES_SHARED(...)
# define TSA_NO_THREAD_SAFETY_ANALYSIS
# define TSA_CAPABILITY(...)
# define TSA_ACQUIRE(...)
# define TSA_TRY_ACQUIRE(...)
# define TSA_RELEASE(...)
# define TSA_ACQUIRE_SHARED(...)
# define TSA_TRY_ACQUIRE_SHARED(...)
# define TSA_RELEASE_SHARED(...)
# define TSA_SUPPRESS_WARNING_FOR_READ(x) (x)
# define TSA_SUPPRESS_WARNING_FOR_WRITE(x) (x)

2
contrib/azure vendored

@ -1 +1 @@
Subproject commit ef75afc075fc71fbcd8fe28dcda3794ae265fd1c
Subproject commit ea8c3044f43f5afa7016d2d580ed201f495d7e94

View File

@ -0,0 +1,11 @@
version: '2.3'
services:
kerberoskdc:
image: clickhouse/kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest}
hostname: kerberoskdc
volumes:
- ${KERBEROS_KDC_DIR}/secrets:/tmp/keytab
- ${KERBEROS_KDC_DIR}/../kerberos_image_config.sh:/config.sh
- /dev/urandom:/dev/random
ports: [88, 749]

View File

@ -5,12 +5,18 @@ FROM ubuntu:22.04
ARG apt_archive="http://archive.ubuntu.com"
RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
RUN apt-get update --yes && env DEBIAN_FRONTEND=noninteractive apt-get install wget unzip git default-jdk maven python3 --yes --no-install-recommends
RUN wget https://github.com/sqlancer/sqlancer/archive/master.zip -O /sqlancer.zip
RUN apt-get update --yes && \
env DEBIAN_FRONTEND=noninteractive apt-get install wget git default-jdk maven python3 --yes --no-install-recommends && \
apt-get clean
# We need to get the repository's HEAD each time despite, so we invalidate layers' cache
ARG CACHE_INVALIDATOR=0
RUN mkdir /sqlancer && \
cd /sqlancer && \
unzip /sqlancer.zip
RUN cd /sqlancer/sqlancer-master && mvn package -DskipTests
wget -q -O- https://github.com/sqlancer/sqlancer/archive/master.tar.gz | \
tar zx -C /sqlancer && \
cd /sqlancer/sqlancer-master && \
mvn package -DskipTests && \
rm -r /root/.m2
COPY run.sh /
COPY process_sqlancer_result.py /

View File

@ -593,7 +593,7 @@ clickhouse-local --structure "test String, res String" -q "SELECT 'failure', tes
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv
# Core dumps
find . -type f -name 'core.*' | while read core; do
find . -type f -maxdepth 1 -name 'core.*' | while read core; do
zstd --threads=0 $core
mv $core.zst /test_output/
done

View File

@ -6,6 +6,8 @@ import argparse
import csv
# TODO: add typing and log files to the fourth column, think about launching
# everything from the python and not bash
def process_result(result_folder):
status = "success"
description = ""

View File

@ -0,0 +1,16 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v22.3.17.13-lts (fcc4de7e805) FIXME as compared to v22.3.16.1190-lts (bb4e0934e5a)
#### Improvement
* Backported in [#45138](https://github.com/ClickHouse/ClickHouse/issues/45138): Allow to use String type instead of Binary in Arrow/Parquet/ORC formats. This PR introduces 3 new settings for it: `output_format_arrow_string_as_string`, `output_format_parquet_string_as_string`, `output_format_orc_string_as_string`. Default value for all settings is `false`. [#37327](https://github.com/ClickHouse/ClickHouse/pull/37327) ([Kruglov Pavel](https://github.com/Avogar)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Improve release scripts [#45074](https://github.com/ClickHouse/ClickHouse/pull/45074) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -22,10 +22,12 @@ To enable Kerberos, one should include `kerberos` section in `config.xml`. This
- `principal` - canonical service principal name that will be acquired and used when accepting security contexts.
- This parameter is optional, if omitted, the default principal will be used.
- `realm` - a realm, that will be used to restrict authentication to only those requests whose initiator's realm matches it.
- This parameter is optional, if omitted, no additional filtering by realm will be applied.
- `keytab` - path to service keytab file.
- This parameter is optional, if omitted, path to service keytab file must be set in `KRB5_KTNAME` environment variable.
Example (goes into `config.xml`):
```xml

View File

@ -207,16 +207,9 @@ Converts a date or date with time to a UInt8 number containing the number of the
Aliases: `DAYOFMONTH`, `DAY`.
## toDayOfWeek(date\[,mode\])
## toDayOfWeek
Converts a date or date with time to a UInt8 number containing the number of the day of the week. The two-argument form of toDayOfWeek() enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or from 1-7. If the mode argument is ommited, the default mode is 0.
| Mode | First day of week | Range |
|------|-------------------|------------------------------------------------|
| 0 | Monday | 1-7, Monday = 1, Tuesday = 2, ..., Sunday = 7 |
| 1 | Monday | 0-6, Monday = 0, Tuesday = 1, ..., Sunday = 6 |
| 2 | Sunday | 0-6, Sunday = 0, Monday = 1, ..., Saturday = 6 |
| 3 | Sunday | 1-7, Sunday = 1, Monday = 2, ..., Saturday = 7 |
Converts a date or date with time to a UInt8 number containing the number of the day of the week (Monday is 1, and Sunday is 7).
Alias: `DAYOFWEEK`.

View File

@ -36,6 +36,18 @@ This query is fully equivalent to using the subquery:
SELECT a, b, c FROM (SELECT ...)
```
## Parameterized View
Parametrized views are similar to normal views, but can be created with parameters which are not resolved immediately. These views can be used with table functions, which specify the name of the view as function name and the parameter values as its arguments.
``` sql
CREATE VIEW view AS SELECT * FROM TABLE WHERE Column1={column1:datatype1} and Column2={column2:datatype2} ...
```
The above creates a view for table which can be used as table function by substituting parameters as shown below.
``` sql
SELECT * FROM view(column1=value1, column2=value2 ...)
```
## Materialized View
``` sql

View File

@ -22,6 +22,9 @@ ClickHouse предоставляет возможность аутентифи
- `realm` — обеспечивает фильтрацию по реалм (realm). Пользователям, чей реалм не совпадает с указанным, будет отказано в аутентификации.
- Это опциональный параметр, при его отсутствии фильтр по реалм применяться не будет.
- `keytab` — задаёт путь к файлу keytab.
- Это опциональный параметр, при его отсутствии путь к файлу keytab должен быть задан в переменной окружения `KRB5_KTNAME`.
Примеры, как должен выглядеть файл `config.xml`:
```xml

View File

@ -10,7 +10,6 @@
#include <optional>
#include <utility>
namespace DB
{
@ -223,6 +222,7 @@ void parseKerberosParams(GSSAcceptorContext::Params & params, const Poco::Util::
params.realm = config.getString("kerberos.realm", "");
params.principal = config.getString("kerberos.principal", "");
params.keytab = config.getString("kerberos.keytab", "");
}
}

View File

@ -6,6 +6,7 @@
#include <mutex>
#include <tuple>
#include <filesystem>
namespace DB
@ -261,6 +262,15 @@ void GSSAcceptorContext::initHandles()
resetHandles();
if (!params.keytab.empty())
{
if (!std::filesystem::exists(params.keytab))
throw Exception("Keytab file not found", ErrorCodes::BAD_ARGUMENTS);
if (krb5_gss_register_acceptor_identity(params.keytab.c_str()))
throw Exception("Failed to register keytab file", ErrorCodes::BAD_ARGUMENTS);
}
if (!params.principal.empty())
{
if (!params.realm.empty())

View File

@ -9,6 +9,7 @@
#if USE_KRB5
# include <gssapi/gssapi.h>
# include <gssapi/gssapi_ext.h>
# include <gssapi/gssapi_krb5.h>
# define MAYBE_NORETURN
#else
# define MAYBE_NORETURN [[noreturn]]
@ -28,6 +29,7 @@ public:
String mechanism = "1.2.840.113554.1.2.2"; // OID: krb5
String principal;
String realm;
String keytab;
};
explicit GSSAcceptorContext(const Params & params_);

View File

@ -606,5 +606,10 @@ if (ENABLE_TESTS)
target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::yaml_cpp)
endif()
if (TARGET ch_contrib::azure_sdk)
target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::azure_sdk)
endif()
add_check(unit_tests_dbms)
endif ()

View File

@ -905,11 +905,51 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
select->where()->children.clear();
select->setExpression(ASTSelectQuery::Expression::WHERE, {});
}
else if (!select->prewhere().get())
{
if (fuzz_rand() % 50 == 0)
{
select->setExpression(ASTSelectQuery::Expression::PREWHERE, select->where()->clone());
if (fuzz_rand() % 2 == 0)
{
select->where()->children.clear();
select->setExpression(ASTSelectQuery::Expression::WHERE, {});
}
}
}
}
else if (fuzz_rand() % 50 == 0)
{
select->setExpression(ASTSelectQuery::Expression::WHERE, getRandomColumnLike());
}
if (select->prewhere().get())
{
if (fuzz_rand() % 50 == 0)
{
select->prewhere()->children.clear();
select->setExpression(ASTSelectQuery::Expression::PREWHERE, {});
}
else if (!select->where().get())
{
if (fuzz_rand() % 50 == 0)
{
select->setExpression(ASTSelectQuery::Expression::WHERE, select->prewhere()->clone());
if (fuzz_rand() % 2 == 0)
{
select->prewhere()->children.clear();
select->setExpression(ASTSelectQuery::Expression::PREWHERE, {});
}
}
}
}
else if (fuzz_rand() % 50 == 0)
{
select->setExpression(ASTSelectQuery::Expression::PREWHERE, getRandomColumnLike());
}
fuzzOrderByList(select->orderBy().get());
fuzz(select->children);

243
src/Common/CancelToken.cpp Normal file
View File

@ -0,0 +1,243 @@
#include <Common/CancelToken.h>
namespace DB
{
namespace ErrorCodes
{
extern const int THREAD_WAS_CANCELED;
}
}
#ifdef OS_LINUX /// Because of futex
#include <base/getThreadId.h>
#include <linux/futex.h>
#include <sys/types.h>
#include <sys/syscall.h>
#include <unistd.h>
namespace DB
{
namespace
{
inline Int64 futexWait(void * address, UInt32 value)
{
return syscall(SYS_futex, address, FUTEX_WAIT_PRIVATE, value, nullptr, nullptr, 0);
}
inline Int64 futexWake(void * address, int count)
{
return syscall(SYS_futex, address, FUTEX_WAKE_PRIVATE, count, nullptr, nullptr, 0);
}
}
void CancelToken::Registry::insert(CancelToken * token)
{
std::lock_guard<std::mutex> lock(mutex);
threads[token->thread_id] = token;
}
void CancelToken::Registry::remove(CancelToken * token)
{
std::lock_guard<std::mutex> lock(mutex);
threads.erase(token->thread_id);
}
void CancelToken::Registry::signal(UInt64 tid)
{
std::lock_guard<std::mutex> lock(mutex);
if (auto it = threads.find(tid); it != threads.end())
it->second->signalImpl();
}
void CancelToken::Registry::signal(UInt64 tid, int code, const String & message)
{
std::lock_guard<std::mutex> lock(mutex);
if (auto it = threads.find(tid); it != threads.end())
it->second->signalImpl(code, message);
}
const std::shared_ptr<CancelToken::Registry> & CancelToken::Registry::instance()
{
static std::shared_ptr<Registry> registry{new Registry()}; // shared_ptr is used to enforce correct destruction order of tokens and registry
return registry;
}
CancelToken::CancelToken()
: state(disabled)
, thread_id(getThreadId())
, registry(Registry::instance())
{
registry->insert(this);
}
CancelToken::~CancelToken()
{
registry->remove(this);
}
void CancelToken::signal(UInt64 tid)
{
Registry::instance()->signal(tid);
}
void CancelToken::signal(UInt64 tid, int code, const String & message)
{
Registry::instance()->signal(tid, code, message);
}
bool CancelToken::wait(UInt32 * address, UInt32 value)
{
chassert((reinterpret_cast<UInt64>(address) & canceled) == 0); // An `address` must be 2-byte aligned
if (value & signaled) // Can happen after spurious wake-up due to cancel of other thread
return true; // Spin-wait unless signal is handled
UInt64 s = state.load();
while (true)
{
if (s & disabled)
{
// Start non-cancelable wait on futex. Spurious wake-up is possible.
futexWait(address, value);
return true; // Disabled - true is forced
}
if (s & canceled)
return false; // Has already been canceled
if (state.compare_exchange_strong(s, reinterpret_cast<UInt64>(address)))
break; // This futex has been "acquired" by this token
}
// Start cancelable wait. Spurious wake-up is possible.
futexWait(address, value);
// "Release" futex and check for cancellation
s = state.load();
while (true)
{
chassert((s & disabled) != disabled); // `disable()` must not be called from another thread
if (s & canceled)
{
if (s == canceled)
break; // Signaled; futex "release" has been done by the signaling thread
else
{
s = state.load();
continue; // To avoid race (may lead to futex destruction) we have to wait for signaling thread to finish
}
}
if (state.compare_exchange_strong(s, 0))
return true; // There was no cancellation; futex "released"
}
// Reset signaled bit
reinterpret_cast<std::atomic<UInt32> *>(address)->fetch_and(~signaled);
return false;
}
void CancelToken::raise()
{
std::unique_lock lock(signal_mutex);
if (exception_code != 0)
throw DB::Exception(
std::exchange(exception_code, 0),
std::exchange(exception_message, {}));
else
throw DB::Exception(ErrorCodes::THREAD_WAS_CANCELED, "Thread was canceled");
}
void CancelToken::notifyOne(UInt32 * address)
{
futexWake(address, 1);
}
void CancelToken::notifyAll(UInt32 * address)
{
futexWake(address, INT_MAX);
}
void CancelToken::signalImpl()
{
signalImpl(0, {});
}
std::mutex CancelToken::signal_mutex;
void CancelToken::signalImpl(int code, const String & message)
{
// Serialize all signaling threads to avoid races due to concurrent signal()/raise() calls
std::unique_lock lock(signal_mutex);
UInt64 s = state.load();
while (true)
{
if (s & canceled)
return; // Already canceled - don't signal twice
if (state.compare_exchange_strong(s, s | canceled))
break; // It is the canceling thread - should deliver signal if necessary
}
exception_code = code;
exception_message = message;
if ((s & disabled) == disabled)
return; // cancellation is disabled - just signal token for later, but don't wake
std::atomic<UInt32> * address = reinterpret_cast<std::atomic<UInt32> *>(s & disabled);
if (address == nullptr)
return; // Thread is currently not waiting on futex - wake-up not required
// Set signaled bit
UInt32 value = address->load();
while (true)
{
if (value & signaled) // Already signaled, just spin-wait until previous signal is handled by waiter
value = address->load();
else if (address->compare_exchange_strong(value, value | signaled))
break;
}
// Wake all threads waiting on `address`, one of them will be canceled and others will get spurious wake-ups
// Woken canceled thread will reset signaled bit
futexWake(address, INT_MAX);
// Signaling thread must remove address from state to notify canceled thread that `futexWake()` is done, thus `wake()` can return.
// Otherwise we may have race condition: signaling thread may try to wake futex that has been already destructed.
state.store(canceled);
}
Cancelable::Cancelable()
{
CancelToken::local().reset();
}
Cancelable::~Cancelable()
{
CancelToken::local().disable();
}
NonCancelable::NonCancelable()
{
CancelToken::local().disable();
}
NonCancelable::~NonCancelable()
{
CancelToken::local().enable();
}
}
#else
namespace DB
{
void CancelToken::raise()
{
throw DB::Exception(ErrorCodes::THREAD_WAS_CANCELED, "Thread was canceled");
}
}
#endif

207
src/Common/CancelToken.h Normal file
View File

@ -0,0 +1,207 @@
#pragma once
#include <base/types.h>
#include <base/defines.h>
#include <Common/Exception.h>
#ifdef OS_LINUX /// Because of futex
#include <atomic>
#include <mutex>
#include <unordered_map>
#include <memory>
namespace DB
{
// Scoped object, enabling thread cancellation (cannot be nested).
// Intended to be used once per cancelable task. It erases any previously held cancellation signal.
// Note that by default thread is not cancelable.
struct Cancelable
{
Cancelable();
~Cancelable();
};
// Scoped object, disabling thread cancellation (cannot be nested; must be inside `Cancelable` region)
struct NonCancelable
{
NonCancelable();
~NonCancelable();
};
// Responsible for synchronization needed to deliver thread cancellation signal.
// Basic building block for cancelable synchronization primitives.
// Allows to perform cancelable wait on memory addresses (think futex)
class CancelToken
{
public:
CancelToken();
CancelToken(const CancelToken &) = delete;
CancelToken(CancelToken &&) = delete;
CancelToken & operator=(const CancelToken &) = delete;
CancelToken & operator=(CancelToken &&) = delete;
~CancelToken();
// Returns token for the current thread
static CancelToken & local()
{
static thread_local CancelToken token;
return token;
}
// Cancelable wait on memory address (futex word).
// Thread will do atomic compare-and-sleep `*address == value`. Waiting will continue until `notify_one()`
// or `notify_all()` will be called with the same `address` or calling thread will be canceled using `signal()`.
// Note that spurious wake-ups are also possible due to cancellation of other waiters on the same `address`.
// WARNING: `address` must be 2-byte aligned and `value` highest bit must be zero.
// Return value:
// true - woken by either notify or spurious wakeup;
// false - iff cancellation signal has been received.
// Implementation details:
// It registers `address` inside token's `state` to allow other threads to wake this thread and deliver cancellation signal.
// Highest bit of `*address` is used for guaranteed delivery of the signal, but is guaranteed to be zero on return due to cancellation.
// Intended to be called only by thread associated with this token.
bool wait(UInt32 * address, UInt32 value);
// Throws `DB::Exception` received from `signal()`. Call it if `wait()` returned false.
// Intended to be called only by thread associated with this token.
[[noreturn]] void raise();
// Regular wake by address (futex word). It does not interact with token in any way. We have it here to complement `wait()`.
// Can be called from any thread.
static void notifyOne(UInt32 * address);
static void notifyAll(UInt32 * address);
// Send cancel signal to thread with specified `tid`.
// If thread was waiting using `wait()` it will be woken up (unless cancellation is disabled).
// Can be called from any thread.
static void signal(UInt64 tid);
static void signal(UInt64 tid, int code, const String & message);
// Flag used to deliver cancellation into memory address to wake a thread.
// Note that most significant bit at `addresses` to be used with `wait()` is reserved.
static constexpr UInt32 signaled = 1u << 31u;
private:
friend struct Cancelable;
friend struct NonCancelable;
// Restores initial state for token to be reused. See `Cancelable` struct.
// Intended to be called only by thread associated with this token.
void reset()
{
state.store(0);
}
// Enable thread cancellation. See `NonCancelable` struct.
// Intended to be called only by thread associated with this token.
void enable()
{
chassert((state.load() & disabled) == disabled);
state.fetch_and(~disabled);
}
// Disable thread cancellation. See `NonCancelable` struct.
// Intended to be called only by thread associated with this token.
void disable()
{
chassert((state.load() & disabled) == 0);
state.fetch_or(disabled);
}
// Singleton. Maps thread IDs to tokens.
struct Registry
{
std::mutex mutex;
std::unordered_map<UInt64, CancelToken*> threads; // By thread ID
void insert(CancelToken * token);
void remove(CancelToken * token);
void signal(UInt64 tid);
void signal(UInt64 tid, int code, const String & message);
static const std::shared_ptr<Registry> & instance();
};
// Cancels this token and wakes thread if necessary.
// Can be called from any thread.
void signalImpl();
void signalImpl(int code, const String & message);
// Lower bit: cancel signal received flag
static constexpr UInt64 canceled = 1;
// Upper bits - possible values:
// 1) all zeros: token is enabed, i.e. wait() call can return false, thread is not waiting on any address;
// 2) all ones: token is disabled, i.e. wait() call cannot be canceled;
// 3) specific `address`: token is enabled and thread is currently waiting on this `address`.
static constexpr UInt64 disabled = ~canceled;
static_assert(sizeof(UInt32 *) == sizeof(UInt64)); // State must be able to hold an address
// All signal handling logic should be globally serialized using this mutex
static std::mutex signal_mutex;
// Cancellation state
alignas(64) std::atomic<UInt64> state;
[[maybe_unused]] char padding[64 - sizeof(state)];
// Cancellation exception
int exception_code;
String exception_message;
// Token is permanently attached to a single thread. There is one-to-one mapping between threads and tokens.
const UInt64 thread_id;
// To avoid `Registry` destruction before last `Token` destruction
const std::shared_ptr<Registry> registry;
};
}
#else
// WARNING: We support cancelable synchronization primitives only on linux for now
namespace DB
{
struct Cancelable
{
Cancelable() = default;
~Cancelable() = default;
};
struct NonCancelable
{
NonCancelable() = default;
~NonCancelable() = default;
};
class CancelToken
{
public:
CancelToken() = default;
CancelToken(const CancelToken &) = delete;
CancelToken(CancelToken &&) = delete;
CancelToken & operator=(const CancelToken &) = delete;
~CancelToken() = default;
static CancelToken & local()
{
static CancelToken token;
return token;
}
bool wait(UInt32 *, UInt32) { return true; }
[[noreturn]] void raise();
static void notifyOne(UInt32 *) {}
static void notifyAll(UInt32 *) {}
static void signal(UInt64) {}
static void signal(UInt64, int, const String &) {}
};
}
#endif

View File

@ -0,0 +1,115 @@
#include <Common/CancelableSharedMutex.h>
#ifdef OS_LINUX /// Because of futex
#include <Common/futex.h>
namespace DB
{
namespace
{
inline bool cancelableWaitUpperFetch(std::atomic<UInt64> & address, UInt64 & value)
{
bool res = CancelToken::local().wait(upperHalfAddress(&address), upperHalf(value));
value = address.load();
return res;
}
inline bool cancelableWaitLowerFetch(std::atomic<UInt64> & address, UInt64 & value)
{
bool res = CancelToken::local().wait(lowerHalfAddress(&address), lowerHalf(value));
value = address.load();
return res;
}
}
CancelableSharedMutex::CancelableSharedMutex()
: state(0)
, waiters(0)
{}
void CancelableSharedMutex::lock()
{
UInt64 value = state.load();
while (true)
{
if (value & writers)
{
waiters++;
if (!cancelableWaitUpperFetch(state, value))
{
waiters--;
CancelToken::local().raise();
}
else
waiters--;
}
else if (state.compare_exchange_strong(value, value | writers))
break;
}
value |= writers;
while (value & readers)
{
if (!cancelableWaitLowerFetch(state, value))
{
state.fetch_and(~writers);
futexWakeUpperAll(state);
CancelToken::local().raise();
}
}
}
bool CancelableSharedMutex::try_lock()
{
UInt64 value = state.load();
return (value & (readers | writers)) == 0 && state.compare_exchange_strong(value, value | writers);
}
void CancelableSharedMutex::unlock()
{
state.fetch_and(~writers);
if (waiters)
futexWakeUpperAll(state);
}
void CancelableSharedMutex::lock_shared()
{
UInt64 value = state.load();
while (true)
{
if (value & writers)
{
waiters++;
if (!cancelableWaitUpperFetch(state, value))
{
waiters--;
CancelToken::local().raise();
}
else
waiters--;
}
else if (state.compare_exchange_strong(value, value + 1)) // overflow is not realistic
break;
}
}
bool CancelableSharedMutex::try_lock_shared()
{
UInt64 value = state.load();
if (!(value & writers) && state.compare_exchange_strong(value, value + 1)) // overflow is not realistic
return true;
return false;
}
void CancelableSharedMutex::unlock_shared()
{
UInt64 value = state.fetch_sub(1) - 1;
if ((value & (writers | readers)) == writers) // If writer is waiting and no more readers
futexWakeLowerOne(state); // Wake writer
}
}
#endif

View File

@ -0,0 +1,64 @@
#pragma once
#include <shared_mutex>
#ifdef OS_LINUX /// Because of futex
#include <Common/CancelToken.h>
#include <base/types.h>
#include <base/defines.h>
#include <atomic>
namespace DB
{
// Reimplementation of `std::shared_mutex` that can interoperate with thread cancellation via `CancelToken::signal()`.
// It has cancellation point on waiting during `lock()` and `shared_lock()`.
// NOTE: It has NO cancellation points on fast code path, when locking does not require waiting.
class TSA_CAPABILITY("CancelableSharedMutex") CancelableSharedMutex
{
public:
CancelableSharedMutex();
~CancelableSharedMutex() = default;
CancelableSharedMutex(const CancelableSharedMutex &) = delete;
CancelableSharedMutex & operator=(const CancelableSharedMutex &) = delete;
// Exclusive ownership
void lock() TSA_ACQUIRE();
bool try_lock() TSA_TRY_ACQUIRE(true);
void unlock() TSA_RELEASE();
// Shared ownership
void lock_shared() TSA_ACQUIRE_SHARED();
bool try_lock_shared() TSA_TRY_ACQUIRE_SHARED(true);
void unlock_shared() TSA_RELEASE_SHARED();
private:
// State 64-bits layout:
// 1b - 31b - 1b - 31b
// signaled - writers - signaled - readers
// 63------------------------------------0
// Two 32-bit words are used for cancelable waiting, so each has its own separate signaled bit
static constexpr UInt64 readers = (1ull << 32ull) - 1ull - CancelToken::signaled;
static constexpr UInt64 readers_signaled = CancelToken::signaled;
static constexpr UInt64 writers = readers << 32ull;
static constexpr UInt64 writers_signaled = readers_signaled << 32ull;
alignas(64) std::atomic<UInt64> state;
std::atomic<UInt32> waiters;
};
}
#else
// WARNING: We support cancelable synchronization primitives only on linux for now
namespace DB
{
using CancelableSharedMutex = std::shared_mutex;
}
#endif

View File

@ -39,15 +39,6 @@ enum class WeekModeFlag : UInt8
};
using YearWeek = std::pair<UInt16, UInt8>;
/// Modes for toDayOfWeek() function.
enum class WeekDayMode
{
WeekStartsMonday1 = 0,
WeekStartsMonday0 = 1,
WeekStartsSunday0 = 2,
WeekStartsSunday1 = 3
};
/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on.
* First time was implemented for OLAPServer, that needed to do billions of such transformations.
*/
@ -628,25 +619,9 @@ public:
template <typename DateOrTime>
inline Int16 toYear(DateOrTime v) const { return lut[toLUTIndex(v)].year; }
/// 1-based, starts on Monday
template <typename DateOrTime>
inline UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; }
template <typename DateOrTime>
inline UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const
{
WeekDayMode mode = check_week_day_mode(week_day_mode);
UInt8 res = toDayOfWeek(v);
bool start_from_sunday = (mode == WeekDayMode::WeekStartsSunday0 || mode == WeekDayMode::WeekStartsSunday1);
bool zero_based = (mode == WeekDayMode::WeekStartsMonday0 || mode == WeekDayMode::WeekStartsSunday0);
if (start_from_sunday)
res = res % 7 + 1;
if (zero_based)
--res;
return res;
}
template <typename DateOrTime>
inline UInt8 toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; }
@ -869,13 +844,6 @@ public:
return week_format;
}
/// Check and change mode to effective.
inline WeekDayMode check_week_day_mode(UInt8 mode) const /// NOLINT
{
return static_cast<WeekDayMode>(mode & 3);
}
/** Calculate weekday from d.
* Returns 0 for monday, 1 for tuesday...
*/

View File

@ -645,6 +645,7 @@
M(674, RESOURCE_NOT_FOUND) \
M(675, CANNOT_PARSE_IPV4) \
M(676, CANNOT_PARSE_IPV6) \
M(677, THREAD_WAS_CANCELED) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -0,0 +1,85 @@
#include <Common/SharedMutex.h>
#ifdef OS_LINUX /// Because of futex
#include <bit>
#include <Common/futex.h>
namespace DB
{
SharedMutex::SharedMutex()
: state(0)
, waiters(0)
{}
void SharedMutex::lock()
{
UInt64 value = state.load();
while (true)
{
if (value & writers)
{
waiters++;
futexWaitUpperFetch(state, value);
waiters--;
}
else if (state.compare_exchange_strong(value, value | writers))
break;
}
value |= writers;
while (value & readers)
futexWaitLowerFetch(state, value);
}
bool SharedMutex::try_lock()
{
UInt64 value = 0;
if (state.compare_exchange_strong(value, writers))
return true;
return false;
}
void SharedMutex::unlock()
{
state.store(0);
if (waiters)
futexWakeUpperAll(state);
}
void SharedMutex::lock_shared()
{
UInt64 value = state.load();
while (true)
{
if (value & writers)
{
waiters++;
futexWaitUpperFetch(state, value);
waiters--;
}
else if (state.compare_exchange_strong(value, value + 1))
break;
}
}
bool SharedMutex::try_lock_shared()
{
UInt64 value = state.load();
if (!(value & writers) && state.compare_exchange_strong(value, value + 1))
return true;
return false;
}
void SharedMutex::unlock_shared()
{
UInt64 value = state.fetch_sub(1) - 1;
if (value == writers)
futexWakeLowerOne(state); // Wake writer
}
}
#endif

52
src/Common/SharedMutex.h Normal file
View File

@ -0,0 +1,52 @@
#pragma once
#include <shared_mutex>
#ifdef OS_LINUX /// Because of futex
#include <base/types.h>
#include <base/defines.h>
#include <atomic>
namespace DB
{
// Faster implementation of `std::shared_mutex` based on a pair of futexes
class TSA_CAPABILITY("SharedMutex") SharedMutex
{
public:
SharedMutex();
~SharedMutex() = default;
SharedMutex(const SharedMutex &) = delete;
SharedMutex & operator=(const SharedMutex &) = delete;
// Exclusive ownership
void lock() TSA_ACQUIRE();
bool try_lock() TSA_TRY_ACQUIRE(true);
void unlock() TSA_RELEASE();
// Shared ownership
void lock_shared() TSA_ACQUIRE_SHARED();
bool try_lock_shared() TSA_TRY_ACQUIRE_SHARED(true);
void unlock_shared() TSA_RELEASE_SHARED();
private:
static constexpr UInt64 readers = (1ull << 32ull) - 1ull; // Lower 32 bits of state
static constexpr UInt64 writers = ~readers; // Upper 32 bits of state
alignas(64) std::atomic<UInt64> state;
std::atomic<UInt32> waiters;
};
}
#else
namespace DB
{
using SharedMutex = std::shared_mutex;
}
#endif

97
src/Common/futex.h Normal file
View File

@ -0,0 +1,97 @@
#pragma once
#ifdef OS_LINUX
#include <base/types.h>
#include <bit>
#include <linux/futex.h>
#include <sys/types.h>
#include <sys/syscall.h>
#include <unistd.h>
namespace DB
{
inline Int64 futexWait(void * address, UInt32 value)
{
return syscall(SYS_futex, address, FUTEX_WAIT_PRIVATE, value, nullptr, nullptr, 0);
}
inline Int64 futexWake(void * address, int count)
{
return syscall(SYS_futex, address, FUTEX_WAKE_PRIVATE, count, nullptr, nullptr, 0);
}
inline void futexWaitFetch(std::atomic<UInt32> & address, UInt32 & value)
{
futexWait(&address, value);
value = address.load();
}
inline void futexWakeOne(std::atomic<UInt32> & address)
{
futexWake(&address, 1);
}
inline void futexWakeAll(std::atomic<UInt32> & address)
{
futexWake(&address, INT_MAX);
}
inline constexpr UInt32 lowerHalf(UInt64 value)
{
return static_cast<UInt32>(value & 0xffffffffull);
}
inline constexpr UInt32 upperHalf(UInt64 value)
{
return static_cast<UInt32>(value >> 32ull);
}
inline UInt32 * lowerHalfAddress(void * address)
{
return reinterpret_cast<UInt32 *>(address) + (std::endian::native == std::endian::big);
}
inline UInt32 * upperHalfAddress(void * address)
{
return reinterpret_cast<UInt32 *>(address) + (std::endian::native == std::endian::little);
}
inline void futexWaitLowerFetch(std::atomic<UInt64> & address, UInt64 & value)
{
futexWait(lowerHalfAddress(&address), lowerHalf(value));
value = address.load();
}
inline void futexWakeLowerOne(std::atomic<UInt64> & address)
{
futexWake(lowerHalfAddress(&address), 1);
}
inline void futexWakeLowerAll(std::atomic<UInt64> & address)
{
futexWake(lowerHalfAddress(&address), INT_MAX);
}
inline void futexWaitUpperFetch(std::atomic<UInt64> & address, UInt64 & value)
{
futexWait(upperHalfAddress(&address), upperHalf(value));
value = address.load();
}
inline void futexWakeUpperOne(std::atomic<UInt64> & address)
{
futexWake(upperHalfAddress(&address), 1);
}
inline void futexWakeUpperAll(std::atomic<UInt64> & address)
{
futexWake(upperHalfAddress(&address), INT_MAX);
}
}
#endif

View File

@ -0,0 +1,371 @@
#include <gtest/gtest.h>
#include <thread>
#include <condition_variable>
#include <shared_mutex>
#include <barrier>
#include <atomic>
#include "Common/Exception.h"
#include <Common/CancelToken.h>
#include <Common/SharedMutex.h>
#include <Common/CancelableSharedMutex.h>
#include <Common/Stopwatch.h>
#include <base/demangle.h>
#include <base/getThreadId.h>
namespace DB
{
namespace ErrorCodes
{
extern const int THREAD_WAS_CANCELED;
}
}
struct NoCancel {};
// for all PerfTests
static constexpr int requests = 512 * 1024;
static constexpr int max_threads = 16;
template <class T, class Status = NoCancel>
void TestSharedMutex()
{
// Test multiple readers can acquire lock
for (int readers = 1; readers <= 128; readers *= 2)
{
T sm;
std::atomic<int> test(0);
std::barrier sync(readers + 1);
std::vector<std::thread> threads;
threads.reserve(readers);
auto reader = [&]
{
[[maybe_unused]] Status status;
std::shared_lock lock(sm);
sync.arrive_and_wait();
test++;
};
for (int i = 0; i < readers; i++)
threads.emplace_back(reader);
{ // writer
[[maybe_unused]] Status status;
sync.arrive_and_wait(); // wait for all reader to acquire lock to avoid blocking them
std::unique_lock lock(sm);
test++;
}
for (auto & thread : threads)
thread.join();
ASSERT_EQ(test, readers + 1);
}
// Test multiple writers cannot acquire lock simultaneously
for (int writers = 1; writers <= 128; writers *= 2)
{
T sm;
int test = 0;
std::barrier sync(writers);
std::vector<std::thread> threads;
threads.reserve(writers);
auto writer = [&]
{
[[maybe_unused]] Status status;
sync.arrive_and_wait();
std::unique_lock lock(sm);
test++;
};
for (int i = 0; i < writers; i++)
threads.emplace_back(writer);
for (auto & thread : threads)
thread.join();
ASSERT_EQ(test, writers);
}
}
template <class T, class Status = NoCancel>
void TestSharedMutexCancelReader()
{
static constexpr int readers = 8;
static constexpr int tasks_per_reader = 32;
T sm;
std::atomic<int> successes(0);
std::atomic<int> cancels(0);
std::barrier sync(readers + 1);
std::barrier cancel_sync(readers / 2 + 1);
std::vector<std::thread> threads;
std::mutex m;
std::vector<UInt64> tids_to_cancel;
threads.reserve(readers);
auto reader = [&] (int reader_id)
{
if (reader_id % 2 == 0)
{
std::unique_lock lock(m);
tids_to_cancel.emplace_back(getThreadId());
}
for (int task = 0; task < tasks_per_reader; task++) {
try
{
[[maybe_unused]] Status status;
sync.arrive_and_wait(); // (A) sync with writer
sync.arrive_and_wait(); // (B) wait for writer to acquire unique_lock
std::shared_lock lock(sm);
successes++;
}
catch (DB::Exception & e)
{
ASSERT_EQ(e.code(), DB::ErrorCodes::THREAD_WAS_CANCELED);
ASSERT_EQ(e.message(), "test");
cancels++;
cancel_sync.arrive_and_wait(); // (C) sync with writer
}
}
};
for (int reader_id = 0; reader_id < readers; reader_id++)
threads.emplace_back(reader, reader_id);
{ // writer
[[maybe_unused]] Status status;
for (int task = 0; task < tasks_per_reader; task++) {
sync.arrive_and_wait(); // (A) wait for readers to finish previous task
ASSERT_EQ(cancels + successes, task * readers);
ASSERT_EQ(cancels, task * readers / 2);
ASSERT_EQ(successes, task * readers / 2);
std::unique_lock lock(sm);
sync.arrive_and_wait(); // (B) sync with readers
//std::unique_lock lock(m); // not needed, already synced using barrier
for (UInt64 tid : tids_to_cancel)
DB::CancelToken::signal(tid, DB::ErrorCodes::THREAD_WAS_CANCELED, "test");
// This sync is crucial. It is needed to hold `lock` long enough.
// It guarantees that every canceled thread will find `sm` blocked by writer, and thus will begin to wait.
// Wait() call is required for cancellation. Otherwise, fastpath acquire w/o wait will not generate exception.
// And this is the desired behaviour.
cancel_sync.arrive_and_wait(); // (C) wait for cancellation to finish, before unlock.
}
}
for (auto & thread : threads)
thread.join();
ASSERT_EQ(successes, tasks_per_reader * readers / 2);
ASSERT_EQ(cancels, tasks_per_reader * readers / 2);
}
template <class T, class Status = NoCancel>
void TestSharedMutexCancelWriter()
{
static constexpr int writers = 8;
static constexpr int tasks_per_writer = 32;
T sm;
std::atomic<int> successes(0);
std::atomic<int> cancels(0);
std::barrier sync(writers);
std::vector<std::thread> threads;
std::mutex m;
std::vector<UInt64> all_tids;
threads.reserve(writers);
auto writer = [&]
{
{
std::unique_lock lock(m);
all_tids.emplace_back(getThreadId());
}
for (int task = 0; task < tasks_per_writer; task++) {
try
{
[[maybe_unused]] Status status;
sync.arrive_and_wait(); // (A) sync all threads before race to acquire the lock
std::unique_lock lock(sm);
successes++;
// Thread that managed to acquire the lock cancels all other waiting writers
//std::unique_lock lock(m); // not needed, already synced using barrier
for (UInt64 tid : all_tids)
{
if (tid != getThreadId())
DB::CancelToken::signal(tid, DB::ErrorCodes::THREAD_WAS_CANCELED, "test");
}
// This sync is crucial. It is needed to hold `lock` long enough.
// It guarantees that every canceled thread will find `sm` blocked, and thus will begin to wait.
// Wait() call is required for cancellation. Otherwise, fastpath acquire w/o wait will not generate exception.
// And this is the desired behaviour.
sync.arrive_and_wait(); // (B) wait for cancellation to finish, before unlock.
}
catch (DB::Exception & e)
{
ASSERT_EQ(e.code(), DB::ErrorCodes::THREAD_WAS_CANCELED);
ASSERT_EQ(e.message(), "test");
cancels++;
sync.arrive_and_wait(); // (B) sync with race winner
}
}
};
for (int writer_id = 0; writer_id < writers; writer_id++)
threads.emplace_back(writer);
for (auto & thread : threads)
thread.join();
ASSERT_EQ(successes, tasks_per_writer);
ASSERT_EQ(cancels, tasks_per_writer * (writers - 1));
}
template <class T, class Status = NoCancel>
void PerfTestSharedMutexReadersOnly()
{
std::cout << "*** " << demangle(typeid(T).name()) << "/" << demangle(typeid(Status).name()) << " ***" << std::endl;
for (int thrs = 1; thrs <= max_threads; thrs *= 2)
{
T sm;
std::vector<std::thread> threads;
threads.reserve(thrs);
auto reader = [&]
{
[[maybe_unused]] Status status;
for (int request = requests / thrs; request; request--)
{
std::shared_lock lock(sm);
}
};
Stopwatch watch;
for (int i = 0; i < thrs; i++)
threads.emplace_back(reader);
for (auto & thread : threads)
thread.join();
double ns = watch.elapsedNanoseconds();
std::cout << "thrs = " << thrs << ":\t" << ns / requests << " ns\t" << requests * 1e9 / ns << " rps" << std::endl;
}
}
template <class T, class Status = NoCancel>
void PerfTestSharedMutexWritersOnly()
{
std::cout << "*** " << demangle(typeid(T).name()) << "/" << demangle(typeid(Status).name()) << " ***" << std::endl;
for (int thrs = 1; thrs <= max_threads; thrs *= 2)
{
int counter = 0;
T sm;
std::vector<std::thread> threads;
threads.reserve(thrs);
auto writer = [&]
{
[[maybe_unused]] Status status;
for (int request = requests / thrs; request; request--)
{
std::unique_lock lock(sm);
ASSERT_TRUE(counter % 2 == 0);
counter++;
std::atomic_signal_fence(std::memory_order::seq_cst); // force compiler to generate two separate increment instructions
counter++;
}
};
Stopwatch watch;
for (int i = 0; i < thrs; i++)
threads.emplace_back(writer);
for (auto & thread : threads)
thread.join();
ASSERT_EQ(counter, requests * 2);
double ns = watch.elapsedNanoseconds();
std::cout << "thrs = " << thrs << ":\t" << ns / requests << " ns\t" << requests * 1e9 / ns << " rps" << std::endl;
}
}
template <class T, class Status = NoCancel>
void PerfTestSharedMutexRW()
{
std::cout << "*** " << demangle(typeid(T).name()) << "/" << demangle(typeid(Status).name()) << " ***" << std::endl;
for (int thrs = 1; thrs <= max_threads; thrs *= 2)
{
int counter = 0;
T sm;
std::vector<std::thread> threads;
threads.reserve(thrs);
auto reader = [&]
{
[[maybe_unused]] Status status;
for (int request = requests / thrs / 2; request; request--)
{
{
std::shared_lock lock(sm);
ASSERT_TRUE(counter % 2 == 0);
}
{
std::unique_lock lock(sm);
ASSERT_TRUE(counter % 2 == 0);
counter++;
std::atomic_signal_fence(std::memory_order::seq_cst); // force compiler to generate two separate increment instructions
counter++;
}
}
};
Stopwatch watch;
for (int i = 0; i < thrs; i++)
threads.emplace_back(reader);
for (auto & thread : threads)
thread.join();
ASSERT_EQ(counter, requests);
double ns = watch.elapsedNanoseconds();
std::cout << "thrs = " << thrs << ":\t" << ns / requests << " ns\t" << requests * 1e9 / ns << " rps" << std::endl;
}
}
TEST(Threading, SharedMutexSmokeCancelableEnabled) { TestSharedMutex<DB::CancelableSharedMutex, DB::Cancelable>(); }
TEST(Threading, SharedMutexSmokeCancelableDisabled) { TestSharedMutex<DB::CancelableSharedMutex>(); }
TEST(Threading, SharedMutexSmokeFast) { TestSharedMutex<DB::SharedMutex>(); }
TEST(Threading, SharedMutexSmokeStd) { TestSharedMutex<std::shared_mutex>(); }
TEST(Threading, PerfTestSharedMutexReadersOnlyCancelableEnabled) { PerfTestSharedMutexReadersOnly<DB::CancelableSharedMutex, DB::Cancelable>(); }
TEST(Threading, PerfTestSharedMutexReadersOnlyCancelableDisabled) { PerfTestSharedMutexReadersOnly<DB::CancelableSharedMutex>(); }
TEST(Threading, PerfTestSharedMutexReadersOnlyFast) { PerfTestSharedMutexReadersOnly<DB::SharedMutex>(); }
TEST(Threading, PerfTestSharedMutexReadersOnlyStd) { PerfTestSharedMutexReadersOnly<std::shared_mutex>(); }
TEST(Threading, PerfTestSharedMutexWritersOnlyCancelableEnabled) { PerfTestSharedMutexWritersOnly<DB::CancelableSharedMutex, DB::Cancelable>(); }
TEST(Threading, PerfTestSharedMutexWritersOnlyCancelableDisabled) { PerfTestSharedMutexWritersOnly<DB::CancelableSharedMutex>(); }
TEST(Threading, PerfTestSharedMutexWritersOnlyFast) { PerfTestSharedMutexWritersOnly<DB::SharedMutex>(); }
TEST(Threading, PerfTestSharedMutexWritersOnlyStd) { PerfTestSharedMutexWritersOnly<std::shared_mutex>(); }
TEST(Threading, PerfTestSharedMutexRWCancelableEnabled) { PerfTestSharedMutexRW<DB::CancelableSharedMutex, DB::Cancelable>(); }
TEST(Threading, PerfTestSharedMutexRWCancelableDisabled) { PerfTestSharedMutexRW<DB::CancelableSharedMutex>(); }
TEST(Threading, PerfTestSharedMutexRWFast) { PerfTestSharedMutexRW<DB::SharedMutex>(); }
TEST(Threading, PerfTestSharedMutexRWStd) { PerfTestSharedMutexRW<std::shared_mutex>(); }
#ifdef OS_LINUX /// These tests require cancellability
TEST(Threading, SharedMutexCancelReaderCancelableEnabled) { TestSharedMutexCancelReader<DB::CancelableSharedMutex, DB::Cancelable>(); }
TEST(Threading, SharedMutexCancelWriterCancelableEnabled) { TestSharedMutexCancelWriter<DB::CancelableSharedMutex, DB::Cancelable>(); }
#endif

View File

@ -126,6 +126,10 @@ ExternalTable::ExternalTable(const boost::program_options::variables_map & exter
void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream)
{
/// After finishing this function we will be ready to receive the next file, for this we clear all the information received.
/// We should use SCOPE_EXIT because read_buffer should be reset correctly if there will be an exception.
SCOPE_EXIT(clear());
const Settings & settings = getContext()->getSettingsRef();
if (settings.http_max_multipart_form_data_size)
@ -167,9 +171,6 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
CompletedPipelineExecutor executor(pipeline);
executor.execute();
/// We are ready to receive the next file, for this we clear all the information received
clear();
}
}

View File

@ -0,0 +1,25 @@
#include <string>
#include <vector>
#include <Common/logger_useful.h>
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
#include <azure/storage/blobs.hpp>
#include <azure/storage/common/internal/xml_wrapper.hpp>
#include <gtest/gtest.h>
TEST(AzureXMLWrapper, TestLeak)
{
std::string str = "<hello>world</hello>";
Azure::Storage::_internal::XmlReader reader(str.c_str(), str.length());
Azure::Storage::_internal::XmlReader reader2(std::move(reader));
Azure::Storage::_internal::XmlReader reader3 = std::move(reader2);
reader3.Read();
}
#endif

View File

@ -7,49 +7,29 @@
namespace fs = std::filesystem;
template <typename T>
DB::DiskPtr createDisk();
template <>
DB::DiskPtr createDisk<DB::DiskLocal>()
DB::DiskPtr createDisk()
{
fs::create_directory("tmp/");
return std::make_shared<DB::DiskLocal>("local_disk", "tmp/", 0);
}
template <typename T>
void destroyDisk(DB::DiskPtr & disk)
{
disk.reset();
}
template <>
void destroyDisk<DB::DiskLocal>(DB::DiskPtr & disk)
{
disk.reset();
fs::remove_all("tmp/");
}
template <typename T>
class DiskTest : public testing::Test
{
public:
void SetUp() override { disk = createDisk<T>(); }
void TearDown() override { destroyDisk<T>(disk); }
void SetUp() override { disk = createDisk(); }
void TearDown() override { destroyDisk(disk); }
DB::DiskPtr disk;
};
using DiskImplementations = testing::Types<DB::DiskLocal>;
TYPED_TEST_SUITE(DiskTest, DiskImplementations);
TYPED_TEST(DiskTest, createDirectories)
TEST_F(DiskTest, createDirectories)
{
this->disk->createDirectories("test_dir1/");
EXPECT_TRUE(this->disk->isDirectory("test_dir1/"));
@ -59,7 +39,7 @@ TYPED_TEST(DiskTest, createDirectories)
}
TYPED_TEST(DiskTest, writeFile)
TEST_F(DiskTest, writeFile)
{
{
std::unique_ptr<DB::WriteBuffer> out = this->disk->writeFile("test_file");
@ -77,7 +57,7 @@ TYPED_TEST(DiskTest, writeFile)
}
TYPED_TEST(DiskTest, readFile)
TEST_F(DiskTest, readFile)
{
{
std::unique_ptr<DB::WriteBuffer> out = this->disk->writeFile("test_file");
@ -112,7 +92,7 @@ TYPED_TEST(DiskTest, readFile)
}
TYPED_TEST(DiskTest, iterateDirectory)
TEST_F(DiskTest, iterateDirectory)
{
this->disk->createDirectories("test_dir/nested_dir/");

View File

@ -3,14 +3,6 @@
#include <Disks/DiskLocal.h>
#include <Disks/IDisk.h>
template <typename T>
DB::DiskPtr createDisk();
template <>
DB::DiskPtr createDisk<DB::DiskLocal>();
template <typename T>
void destroyDisk(DB::DiskPtr & disk);
template <>
void destroyDisk<DB::DiskLocal>(DB::DiskPtr & disk);

View File

@ -3,7 +3,7 @@
#include <Disks/IDisk.h>
TEST(DiskTest, parentPath)
TEST(DiskPathTest, parentPath)
{
EXPECT_EQ("", DB::parentPath("test_dir/"));
EXPECT_EQ("test_dir/", DB::parentPath("test_dir/nested_dir/"));
@ -11,7 +11,7 @@ TEST(DiskTest, parentPath)
}
TEST(DiskTest, fileName)
TEST(DiskPathTest, fileName)
{
EXPECT_EQ("test_file", DB::fileName("test_file"));
EXPECT_EQ("nested_file", DB::fileName("test_dir/nested_file"));

View File

@ -786,21 +786,21 @@ struct ToDayOfWeekImpl
{
static constexpr auto name = "toDayOfWeek";
static inline UInt8 execute(Int64 t, UInt8 week_day_mode, const DateLUTImpl & time_zone)
static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(t, week_day_mode);
return time_zone.toDayOfWeek(t);
}
static inline UInt8 execute(UInt32 t, UInt8 week_day_mode, const DateLUTImpl & time_zone)
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(t, week_day_mode);
return time_zone.toDayOfWeek(t);
}
static inline UInt8 execute(Int32 d, UInt8 week_day_mode, const DateLUTImpl & time_zone)
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(ExtendedDayNum(d), week_day_mode);
return time_zone.toDayOfWeek(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, UInt8 week_day_mode, const DateLUTImpl & time_zone)
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(DayNum(d), week_day_mode);
return time_zone.toDayOfWeek(DayNum(d));
}
using FactorTransform = ToMondayImpl;

View File

@ -225,8 +225,8 @@ public:
}
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeWeekNumImpl<ResultPrecision::Extended>>>)
{
auto x_day_of_week = TransformDateTime64<ToDayOfWeekImpl>(transform_x.getScaleMultiplier()).execute(x, 0, timezone_x);
auto y_day_of_week = TransformDateTime64<ToDayOfWeekImpl>(transform_y.getScaleMultiplier()).execute(y, 0, timezone_y);
auto x_day_of_week = TransformDateTime64<ToDayOfWeekImpl>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
auto y_day_of_week = TransformDateTime64<ToDayOfWeekImpl>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
if ((x_day_of_week > y_day_of_week)
|| ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour))
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)

View File

@ -276,7 +276,7 @@ private:
{
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
{
const auto day = ToDayOfWeekImpl::execute(source, 0, timezone);
const auto day = ToDayOfWeekImpl::execute(source, timezone);
static constexpr std::string_view day_names[] =
{
"Monday",

View File

@ -344,13 +344,13 @@ private:
static size_t mysqlDayOfWeek(char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone)
{
*dest = '0' + ToDayOfWeekImpl::execute(source, 0, timezone);
*dest = '0' + ToDayOfWeekImpl::execute(source, timezone);
return 1;
}
static size_t mysqlDayOfWeek0To6(char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone)
{
auto day = ToDayOfWeekImpl::execute(source, 0, timezone);
auto day = ToDayOfWeekImpl::execute(source, timezone);
*dest = '0' + (day == 7 ? 0 : day);
return 1;
}
@ -499,13 +499,13 @@ private:
static size_t jodaDayOfWeek1Based(size_t min_represent_digits, char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone)
{
auto week_day = ToDayOfWeekImpl::execute(source, 0, timezone);
auto week_day = ToDayOfWeekImpl::execute(source, timezone);
return writeNumberWithPadding(dest, week_day, min_represent_digits);
}
static size_t jodaDayOfWeekText(size_t min_represent_digits, char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone)
{
auto week_day = ToDayOfWeekImpl::execute(source, 0, timezone);
auto week_day = ToDayOfWeekImpl::execute(source, timezone);
if (week_day == 7)
week_day = 0;

View File

@ -1,12 +1,13 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionCustomWeekToSomething.h>
namespace DB
{
using FunctionToDayOfWeek = FunctionCustomWeekToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
REGISTER_FUNCTION(ToDayOfWeek)
{

View File

@ -24,6 +24,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
@ -38,6 +39,7 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTQueryParameter.h>
#include <Processors/QueryPlan/QueryPlan.h>
@ -53,6 +55,7 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Functions/UserDefined/UserDefinedExecutableFunctionFactory.h>
#include <Parsers/QueryParameterVisitor.h>
namespace DB
@ -535,7 +538,8 @@ ActionsMatcher::Data::Data(
bool only_consts_,
bool create_source_for_in_,
AggregationKeysInfo aggregation_keys_info_,
bool build_expression_with_window_functions_)
bool build_expression_with_window_functions_,
bool is_create_parameterized_view_)
: WithContext(context_)
, set_size_limit(set_size_limit_)
, subquery_depth(subquery_depth_)
@ -549,6 +553,7 @@ ActionsMatcher::Data::Data(
, actions_stack(std::move(actions_dag), context_)
, aggregation_keys_info(aggregation_keys_info_)
, build_expression_with_window_functions(build_expression_with_window_functions_)
, is_create_parameterized_view(is_create_parameterized_view_)
, next_unique_suffix(actions_stack.getLastActions().getOutputs().size() + 1)
{
}
@ -762,8 +767,9 @@ std::optional<NameAndTypePair> ActionsMatcher::getNameAndTypeFromAST(const ASTPt
return NameAndTypePair(child_column_name, node->result_type);
if (!data.only_consts)
throw Exception("Unknown identifier: " + child_column_name + "; there are columns: " + data.actions_stack.dumpNames(),
ErrorCodes::UNKNOWN_IDENTIFIER);
throw Exception(
"Unknown identifier: " + child_column_name + "; there are columns: " + data.actions_stack.dumpNames(),
ErrorCodes::UNKNOWN_IDENTIFIER);
return {};
}
@ -1120,6 +1126,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
const auto * function = child->as<ASTFunction>();
const auto * identifier = child->as<ASTTableIdentifier>();
const auto * query_parameter = child->as<ASTQueryParameter>();
if (function && function->name == "lambda")
{
/// If the argument is a lambda expression, just remember its approximate type.
@ -1200,6 +1207,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
argument_types.push_back(column.type);
argument_names.push_back(column.name);
}
else if (data.is_create_parameterized_view && query_parameter)
{
const auto data_type = DataTypeFactory::instance().get(query_parameter->type);
ColumnWithTypeAndName column(data_type,query_parameter->getColumnName());
data.addColumn(column);
argument_types.push_back(data_type);
argument_names.push_back(query_parameter->name);
}
else
{
/// If the argument is not a lambda expression, call it recursively and find out its type.

View File

@ -134,6 +134,7 @@ public:
ScopeStack actions_stack;
AggregationKeysInfo aggregation_keys_info;
bool build_expression_with_window_functions;
bool is_create_parameterized_view;
/*
* Remember the last unique column suffix to avoid quadratic behavior
@ -154,7 +155,8 @@ public:
bool only_consts_,
bool create_source_for_in_,
AggregationKeysInfo aggregation_keys_info_,
bool build_expression_with_window_functions_ = false);
bool build_expression_with_window_functions_ = false,
bool is_create_parameterized_view_ = false);
/// Does result of the calculation already exists in the block.
bool hasColumn(const String & column_name) const;

View File

@ -106,6 +106,9 @@
#include <Interpreters/Cache/FileCacheFactory.h>
#include <filesystem>
#include <re2/re2.h>
#include <Storages/StorageView.h>
#include <Parsers/ASTFunction.h>
#include <base/find_symbols.h>
#include <Interpreters/Cache/FileCache.h>
@ -151,6 +154,7 @@ namespace ErrorCodes
extern const int INVALID_SETTING_VALUE;
extern const int UNKNOWN_READ_METHOD;
extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_FUNCTION;
}
@ -1319,14 +1323,49 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String
StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint)
{
ASTFunction * function = assert_cast<ASTFunction *>(table_expression.get());
String database_name = getCurrentDatabase();
String table_name = function->name;
if (function->is_compound_name)
{
std::vector<std::string> parts;
splitInto<'.'>(parts, function->name);
if (parts.size() == 2)
{
database_name = parts[0];
table_name = parts[1];
}
}
StoragePtr table = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, getQueryContext());
if (table)
{
if (table.get()->isView() && table->as<StorageView>()->isParameterizedView())
{
function->prefer_subquery_to_function_formatting = true;
return table;
}
}
auto hash = table_expression->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
StoragePtr & res = table_function_results[key];
if (!res)
{
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this());
TableFunctionPtr table_function_ptr;
try
{
table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this());
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_FUNCTION)
{
e.addMessage(" or incorrect parameterized view");
}
throw;
}
if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable())
{
const auto & structure_hint = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns();
@ -1397,10 +1436,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const
key = toString(new_hash.first) + '_' + toString(new_hash.second);
table_function_results[key] = res;
}
return res;
}
return res;
}

View File

@ -75,6 +75,7 @@
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Parsers/formatAST.h>
#include <Parsers/QueryParameterVisitor.h>
namespace DB
{
@ -158,11 +159,13 @@ ExpressionAnalyzer::ExpressionAnalyzer(
size_t subquery_depth_,
bool do_global,
bool is_explain,
PreparedSetsPtr prepared_sets_)
PreparedSetsPtr prepared_sets_,
bool is_create_parameterized_view_)
: WithContext(context_)
, query(query_), settings(getContext()->getSettings())
, subquery_depth(subquery_depth_)
, syntax(syntax_analyzer_result_)
, is_create_parameterized_view(is_create_parameterized_view_)
{
/// Cache prepared sets because we might run analysis multiple times
if (prepared_sets_)
@ -554,7 +557,9 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_
false /* no_makeset */,
only_consts,
!isRemoteStorage() /* create_source_for_in */,
getAggregationKeysInfo());
getAggregationKeysInfo(),
false /* build_expression_with_window_functions */,
is_create_parameterized_view);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
@ -573,7 +578,9 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGP
true /* no_makeset */,
only_consts,
!isRemoteStorage() /* create_source_for_in */,
getAggregationKeysInfo());
getAggregationKeysInfo(),
false /* build_expression_with_window_functions */,
is_create_parameterized_view);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
@ -594,7 +601,9 @@ void ExpressionAnalyzer::getRootActionsForHaving(
false /* no_makeset */,
only_consts,
true /* create_source_for_in */,
getAggregationKeysInfo());
getAggregationKeysInfo(),
false /* build_expression_with_window_functions */,
is_create_parameterized_view);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
@ -1511,6 +1520,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain,
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
getRootActionsForHaving(select_query->having(), only_types, step.actions());
step.addRequiredOutput(select_query->having()->getColumnName());
return true;
@ -1961,7 +1971,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
ExpressionActions(
before_where,
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
auto & column_elem
= before_where_sample.getByName(query.where()->getColumnName());
/// If the filter column is a constant, record it.
if (column_elem.column)
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);

View File

@ -158,13 +158,15 @@ protected:
size_t subquery_depth_,
bool do_global_,
bool is_explain_,
PreparedSetsPtr prepared_sets_);
PreparedSetsPtr prepared_sets_,
bool is_create_parameterized_view_ = false);
ASTPtr query;
const ExtractedSettings settings;
size_t subquery_depth;
TreeRewriterResultPtr syntax;
bool is_create_parameterized_view;
const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; }
@ -318,7 +320,8 @@ public:
options_.subquery_depth,
do_global_,
options_.is_explain,
prepared_sets_)
prepared_sets_,
options_.is_create_parameterized_view)
, metadata_snapshot(metadata_snapshot_)
, required_result_columns(required_result_columns_)
, query_options(options_)

View File

@ -886,20 +886,20 @@ public:
const auto & lhs = lhs_block.getByPosition(i);
const auto & rhs = rhs_block.getByPosition(i);
if (lhs.name != rhs.name)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
lhs_block.dumpStructure(), rhs_block.dumpStructure());
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})",
lhs_block.dumpStructure(), rhs_block.dumpStructure(), lhs.name, rhs.name);
const auto & ltype = recursiveRemoveLowCardinality(lhs.type);
const auto & rtype = recursiveRemoveLowCardinality(rhs.type);
if (!ltype->equals(*rtype))
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
lhs_block.dumpStructure(), rhs_block.dumpStructure());
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})",
lhs_block.dumpStructure(), rhs_block.dumpStructure(), ltype->getName(), rtype->getName());
const auto & lcol = recursiveRemoveLowCardinality(lhs.column);
const auto & rcol = recursiveRemoveLowCardinality(rhs.column);
if (lcol->getDataType() != rcol->getDataType())
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
lhs_block.dumpStructure(), rhs_block.dumpStructure());
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})",
lhs_block.dumpStructure(), rhs_block.dumpStructure(), lcol->getDataType(), rcol->getDataType());
}
}

View File

@ -719,7 +719,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
}
else if (create.select)
{
Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), getContext());
Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), getContext(), false /* is_subquery */, create.isParameterizedView());
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
}
else if (create.as_table_function)

View File

@ -72,7 +72,7 @@ namespace
if (query_info.view_query)
{
ASTPtr tmp;
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp);
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp, query_info.is_parameterized_view);
}
}
};

View File

@ -92,9 +92,9 @@
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/checkStackSize.h>
#include <Common/scope_guard_safe.h>
#include <Parsers/FunctionParameterValuesVisitor.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -505,13 +505,41 @@ InterpreterSelectQuery::InterpreterSelectQuery(
{
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
ASTPtr view_table;
NameToNameMap parameter_values;
NameToNameMap parameter_types;
if (view)
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot);
{
query_info.is_parameterized_view = view->isParameterizedView();
/// We need to fetch the parameters set for SELECT ... FROM parameterized_view(<params>) before the query is replaced.
/// replaceWithSubquery replaces the function child and adds the subquery in its place.
/// the parameters are children of function child, if function (which corresponds to parametrised view and has
/// parameters in its arguments: `parametrised_view(<params>)`) is replaced the parameters are also gone from tree
/// So we need to get the parameters before they are removed from the tree
/// and after query is replaced, we use these parameters to substitute in the parameterized view query
if (query_info.is_parameterized_view)
{
parameter_values = analyzeFunctionParamValues(query_ptr);
view->setParameterValues(parameter_values);
parameter_types = view->getParameterValues();
}
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot, view->isParameterizedView());
if (query_info.is_parameterized_view)
{
view->replaceQueryParametersIfParametrizedView(query_ptr);
}
}
syntax_analyzer_result = TreeRewriter(context).analyzeSelect(
query_ptr,
TreeRewriterResult(source_header.getNamesAndTypesList(), storage, storage_snapshot),
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
options,
joined_tables.tablesWithColumns(),
required_result_column_names,
table_join,
query_info.is_parameterized_view,
parameter_values,
parameter_types);
query_info.syntax_analyzer_result = syntax_analyzer_result;
context->setDistributed(syntax_analyzer_result->is_remote_storage);
@ -638,7 +666,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
query_info.filter_asts.push_back(query_info.additional_filter_ast);
}
source_header = storage_snapshot->getSampleBlockForColumns(required_columns);
source_header = storage_snapshot->getSampleBlockForColumns(required_columns, parameter_values);
}
/// Calculate structure of the result.
@ -2041,7 +2069,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
}
auto syntax_result
= TreeRewriter(context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, storage_snapshot);
= TreeRewriter(context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, storage_snapshot, options.is_create_parameterized_view);
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, context).getActionsDAG(true);
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.

View File

@ -258,12 +258,14 @@ InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast
InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default;
Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, ContextPtr context_, bool is_subquery)
Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, ContextPtr context_, bool is_subquery, bool is_create_parameterized_view)
{
if (!context_->hasQueryContext())
{
if (is_subquery)
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock();
else if (is_create_parameterized_view)
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze()).getSampleBlock();
else
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
}
@ -281,6 +283,12 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
return cache[key]
= InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock();
}
else if (is_create_parameterized_view)
{
return cache[key]
= InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze())
.getSampleBlock();
}
else
{
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();

View File

@ -41,7 +41,8 @@ public:
static Block getSampleBlock(
const ASTPtr & query_ptr_,
ContextPtr context_,
bool is_subquery = false);
bool is_subquery = false,
bool is_create_parameterized_view = false);
void ignoreWithTotals() override;

View File

@ -130,6 +130,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
}
}
void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data)
{
/// normalize JOIN ON section
@ -265,7 +266,10 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
else if (auto * node_select = ast->as<ASTSelectQuery>())
visit(*node_select, ast, data);
else if (auto * node_param = ast->as<ASTQueryParameter>())
throw Exception("Query parameter " + backQuote(node_param->name) + " was not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER);
{
if (!data.is_create_parameterized_view)
throw Exception("Query parameter " + backQuote(node_param->name) + " was not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER);
}
else if (auto * node_function = ast->as<ASTFunction>())
if (node_function->parameters)
visit(node_function->parameters, data);

View File

@ -13,6 +13,7 @@ class ASTSelectQuery;
class ASTIdentifier;
struct ASTTablesInSelectQueryElement;
class Context;
class ASTQueryParameter;
class QueryNormalizer
@ -42,6 +43,7 @@ public:
Aliases & aliases;
const NameSet & source_columns_set;
ExtractedSettings settings;
NameSet query_parameters;
/// tmp data
size_t level;
@ -52,14 +54,16 @@ public:
/// It's Ok to have "c + 1 AS c" in queries, but not in table definition
const bool allow_self_aliases; /// for constructs like "SELECT column + 1 AS column"
bool is_create_parameterized_view;
Data(Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_)
Data(Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_, bool is_create_parameterized_view_ = false)
: aliases(aliases_)
, source_columns_set(source_columns_set_)
, settings(settings_)
, level(0)
, ignore_alias(ignore_alias_)
, allow_self_aliases(allow_self_aliases_)
, is_create_parameterized_view(is_create_parameterized_view_)
{}
};

View File

@ -1,51 +0,0 @@
#include <Interpreters/QueryParameterVisitor.h>
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
namespace DB
{
class QueryParameterVisitor
{
public:
explicit QueryParameterVisitor(NameSet & parameters_name)
: query_parameters(parameters_name)
{
}
void visit(const ASTPtr & ast)
{
for (const auto & child : ast->children)
{
if (const auto & query_parameter = child->as<ASTQueryParameter>())
visitQueryParameter(*query_parameter);
else
visit(child);
}
}
private:
NameSet & query_parameters;
void visitQueryParameter(const ASTQueryParameter & query_parameter)
{
query_parameters.insert(query_parameter.name);
}
};
NameSet analyzeReceiveQueryParams(const std::string & query)
{
NameSet query_params;
const char * query_begin = query.data();
const char * query_end = query.data() + query.size();
ParserQuery parser(query_end);
ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0);
QueryParameterVisitor(query_params).visit(extract_query_ast);
return query_params;
}
}

View File

@ -50,6 +50,7 @@ struct SelectQueryOptions
bool with_all_cols = false; /// asterisk include materialized and aliased columns
bool settings_limit_offset_done = false;
bool is_explain = false; /// The value is true if it's explain statement.
bool is_create_parameterized_view = false;
/// These two fields are used to evaluate shardNum() and shardCount() function when
/// prefer_localhost_replica == 1 and local instance is selected. They are needed because local
@ -77,6 +78,13 @@ struct SelectQueryOptions
return out;
}
SelectQueryOptions createParameterizedView() const
{
SelectQueryOptions out = *this;
out.is_create_parameterized_view = true;
return out;
}
SelectQueryOptions & analyze(bool dry_run = true)
{
only_analyze = dry_run;

View File

@ -458,16 +458,6 @@ TableJoin::createConvertingActions(
LOG_DEBUG(&Poco::Logger::get("TableJoin"), "{} JOIN converting actions: empty", side);
return;
}
auto format_cols = [](const auto & cols) -> std::string
{
std::vector<std::string> str_cols;
str_cols.reserve(cols.size());
for (const auto & col : cols)
str_cols.push_back(fmt::format("'{}': {}", col.name, col.type->getName()));
return fmt::format("[{}]", fmt::join(str_cols, ", "));
};
LOG_DEBUG(&Poco::Logger::get("TableJoin"), "{} JOIN converting actions: {} -> {}",
side, format_cols(dag->getRequiredColumns()), format_cols(dag->getResultColumns()));
};
log_actions("Left", left_converting_actions);
log_actions("Right", right_converting_actions);

View File

@ -20,6 +20,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTColumnsTransformers.h>
#include <Storages/StorageView.h>
namespace DB
@ -249,7 +250,17 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
for (const auto & column : *cols)
{
if (first_table || !data.join_using_columns.contains(column.name))
addIdentifier(columns, table.table, column.name);
{
std::string column_name = column.name;
/// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters
/// and SELECT is used with substitution of these query parameters )
if (!data.parameter_values.empty())
column_name
= StorageView::replaceQueryParameterWithValue(column_name, data.parameter_values, data.parameter_types);
addIdentifier(columns, table.table, column_name);
}
}
}
first_table = false;

View File

@ -28,11 +28,15 @@ public:
const TablesWithColumns & tables;
std::unordered_set<String> join_using_columns;
bool has_columns;
NameToNameMap parameter_values;
NameToNameMap parameter_types;
Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true)
Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true, const NameToNameMap & parameter_values_ = {}, const NameToNameMap & parameter_types_ = {})
: source_columns(source_columns_)
, tables(tables_)
, has_columns(has_columns_)
, parameter_values(parameter_values_)
, parameter_types(parameter_types_)
{}
bool hasColumn(const String & name) const { return source_columns.count(name); }

View File

@ -43,6 +43,7 @@
#include <Parsers/ASTInterpolateElement.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTCreateQuery.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeNullable.h>
@ -52,6 +53,7 @@
#include <Storages/IStorage.h>
#include <Storages/StorageJoin.h>
#include <Common/checkStackSize.h>
#include <Storages/StorageView.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
@ -361,10 +363,11 @@ using ReplacePositionalArgumentsVisitor = InDepthNodeVisitor<OneTypeMatcher<Repl
/// Expand asterisks and qualified asterisks with column names.
/// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer.
void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query, const NameSet & source_columns_set,
const TablesWithColumns & tables_with_columns)
const TablesWithColumns & tables_with_columns, const NameToNameMap & parameter_values = {},
const NameToNameMap & parameter_types = {})
{
LogAST log;
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns);
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns, true/* has_columns */, parameter_values, parameter_types);
TranslateQualifiedNamesVisitor visitor(visitor_data, log.stream());
visitor.visit(query);
@ -1297,7 +1300,10 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
const SelectQueryOptions & select_options,
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns,
const Names & required_result_columns,
std::shared_ptr<TableJoin> table_join) const
std::shared_ptr<TableJoin> table_join,
bool is_parameterized_view,
const NameToNameMap parameter_values,
const NameToNameMap parameter_types) const
{
auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
@ -1335,7 +1341,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix());
}
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns, parameter_values, parameter_types);
/// Optimizes logical expressions.
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
@ -1347,7 +1353,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
all_source_columns_set.insert(name);
}
normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true, getContext());
normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true, getContext(), select_options.is_create_parameterized_view);
// expand GROUP BY ALL
if (select_query->group_by_all)
@ -1389,7 +1395,18 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query);
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
/// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters
/// and SELECT is used with substitution of these query parameters )
/// the replaced column names will be used in the next steps
if (is_parameterized_view)
{
for (auto & column : result.source_columns)
column.name = StorageView::replaceQueryParameterWithValue(column.name, parameter_values, parameter_types);
}
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames();
/// rewrite filters for select query, must go after getArrayJoinedColumns
@ -1450,7 +1467,8 @@ TreeRewriterResultPtr TreeRewriter::analyze(
const StorageSnapshotPtr & storage_snapshot,
bool allow_aggregations,
bool allow_self_aliases,
bool execute_scalar_subqueries) const
bool execute_scalar_subqueries,
bool is_create_parameterized_view) const
{
if (query->as<ASTSelectQuery>())
throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR);
@ -1459,7 +1477,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
TreeRewriterResult result(source_columns, storage, storage_snapshot, false);
normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases, getContext());
normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases, getContext(), is_create_parameterized_view);
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
executeScalarSubqueries(query, getContext(), 0, result.scalars, result.local_scalars, !execute_scalar_subqueries);
@ -1488,7 +1506,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
}
void TreeRewriter::normalize(
ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_)
ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view)
{
if (!UserDefinedSQLFunctionFactory::instance().empty())
UserDefinedSQLFunctionVisitor::visit(query);
@ -1563,7 +1581,7 @@ void TreeRewriter::normalize(
FunctionNameNormalizer().visit(query.get());
/// Common subexpression elimination. Rewrite rules.
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, ignore_alias, settings, allow_self_aliases);
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, ignore_alias, settings, allow_self_aliases, is_create_parameterized_view);
QueryNormalizer(normalizer_data).visit(query);
optimizeGroupingSets(query);

View File

@ -119,7 +119,8 @@ public:
const StorageSnapshotPtr & storage_snapshot = {},
bool allow_aggregations = false,
bool allow_self_aliases = true,
bool execute_scalar_subqueries = true) const;
bool execute_scalar_subqueries = true,
bool is_create_parameterized_view = false) const;
/// Analyze and rewrite select query
TreeRewriterResultPtr analyzeSelect(
@ -128,10 +129,13 @@ public:
const SelectQueryOptions & select_options = {},
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns = {},
const Names & required_result_columns = {},
std::shared_ptr<TableJoin> table_join = {}) const;
std::shared_ptr<TableJoin> table_join = {},
bool is_parameterized_view = false,
const NameToNameMap parameter_values = {},
const NameToNameMap parameter_types = {}) const;
private:
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_);
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false);
};
}

View File

@ -395,8 +395,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (const auto * insert_query = ast->as<ASTInsertQuery>(); insert_query && insert_query->data)
query_end = insert_query->data;
bool is_create_parameterized_view = false;
if (const auto * create_query = ast->as<ASTCreateQuery>())
is_create_parameterized_view = create_query->isParameterizedView();
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
if (context->hasQueryParameters())
if (!is_create_parameterized_view && context->hasQueryParameters())
{
ReplaceQueryParameterVisitor visitor(context->getQueryParameters());
visitor.visit(ast);

View File

@ -443,4 +443,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
}
}
bool ASTCreateQuery::isParameterizedView() const
{
if (is_ordinary_view && select && select->hasQueryParameters())
return true;
return false;
}
}

View File

@ -25,7 +25,6 @@ public:
IAST * ttl_table = nullptr;
ASTSetQuery * settings = nullptr;
String getID(char) const override { return "Storage definition"; }
ASTPtr clone() const override;
@ -119,6 +118,8 @@ public:
bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view || is_window_view; }
bool isParameterizedView() const;
QueryKind getQueryKind() const override { return QueryKind::Create; }
protected:

View File

@ -26,6 +26,10 @@ public:
bool is_lambda_function = false;
/// This field is updated in executeTableFunction if its a parameterized_view
/// and used in ASTTablesInSelectQuery::FormatImpl for EXPLAIN SYNTAX of SELECT parameterized view
bool prefer_subquery_to_function_formatting = false;
// We have to make these fields ASTPtr because this is what the visitors
// expect. Some of them take const ASTPtr & (makes no sense), and some
// take ASTPtr & and modify it. I don't understand how the latter is
@ -67,6 +71,9 @@ public:
std::string getWindowDescription() const;
/// This is used for parameterized view, to identify if name is 'db.view'
bool is_compound_name = false;
bool hasSecretParts() const override;
protected:

View File

@ -7,7 +7,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/StorageID.h>
#include <IO/Operators.h>
#include <Parsers/QueryParameterVisitor.h>
namespace DB
{
@ -479,4 +479,14 @@ void ASTSelectQuery::setFinal() // NOLINT method can be made const
tables_element.table_expression->as<ASTTableExpression &>().final = true;
}
bool ASTSelectQuery::hasQueryParameters() const
{
if (!has_query_parameters.has_value())
{
has_query_parameters = !analyzeReceiveQueryParams(std::make_shared<ASTSelectQuery>(*this)).empty();
}
return has_query_parameters.value();
}
}

View File

@ -3,7 +3,6 @@
#include <Parsers/IAST.h>
#include <Core/Names.h>
namespace DB
{
@ -143,6 +142,7 @@ public:
void setFinal();
QueryKind getQueryKind() const override { return QueryKind::Select; }
bool hasQueryParameters() const;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
@ -150,6 +150,11 @@ protected:
private:
std::unordered_map<Expression, size_t> positions;
/// This variable is optional as we want to set it on the first call to hasQueryParameters
/// and return the same variable on future calls to hasQueryParameters
/// its mutable as we set it in const function
mutable std::optional<bool> has_query_parameters;
ASTPtr & getExpression(Expression expr);
};

View File

@ -3,6 +3,7 @@
#include <Common/typeid_cast.h>
#include <Parsers/SelectUnionMode.h>
#include <IO/Operators.h>
#include <Parsers/ASTSelectQuery.h>
#include <iostream>
@ -92,4 +93,25 @@ bool ASTSelectWithUnionQuery::hasNonDefaultUnionMode() const
|| set_of_modes.contains(SelectUnionMode::EXCEPT_DISTINCT);
}
bool ASTSelectWithUnionQuery::hasQueryParameters() const
{
if (!has_query_parameters.has_value())
{
for (const auto & child : list_of_selects->children)
{
if (auto * select_node = child->as<ASTSelectQuery>())
{
if (select_node->hasQueryParameters())
{
has_query_parameters = true;
return has_query_parameters.value();
}
}
}
has_query_parameters = false;
}
return has_query_parameters.value();
}
}

View File

@ -31,6 +31,15 @@ public:
/// Consider any mode other than ALL as non-default.
bool hasNonDefaultUnionMode() const;
bool hasQueryParameters() const;
private:
/// This variable is optional as we want to set it on the first call to hasQueryParameters
/// and return the same variable on future calls to hasQueryParameters
/// its mutable as we set it in const function
mutable std::optional<bool> has_query_parameters;
};
}

View File

@ -3,6 +3,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Common/SipHash.h>
#include <IO/Operators.h>
#include <Parsers/ASTFunction.h>
namespace DB
@ -112,7 +113,7 @@ void ASTTableExpression::formatImpl(const FormatSettings & settings, FormatState
settings.ostr << " ";
database_and_table_name->formatImpl(settings, state, frame);
}
else if (table_function)
else if (table_function && !(table_function->as<ASTFunction>()->prefer_subquery_to_function_formatting && subquery))
{
settings.ostr << " ";
table_function->formatImpl(settings, state, frame);

View File

@ -846,8 +846,8 @@ public:
class FunctionLayer : public Layer
{
public:
explicit FunctionLayer(String function_name_, bool allow_function_parameters_ = true)
: function_name(function_name_), allow_function_parameters(allow_function_parameters_){}
explicit FunctionLayer(String function_name_, bool allow_function_parameters_ = true, bool is_compound_name_ = false)
: function_name(function_name_), allow_function_parameters(allow_function_parameters_), is_compound_name(is_compound_name_){}
bool parse(IParser::Pos & pos, Expected & expected, Action & action) override
{
@ -988,6 +988,7 @@ public:
function_name += "Distinct";
auto function_node = makeASTFunction(function_name, std::move(elements));
function_node->is_compound_name = is_compound_name;
if (parameters)
{
@ -1043,6 +1044,7 @@ private:
ASTPtr parameters;
bool allow_function_parameters;
bool is_compound_name;
};
/// Layer for priority brackets and tuple function
@ -2100,7 +2102,7 @@ std::unique_ptr<Layer> getFunctionLayer(ASTPtr identifier, bool is_table_functio
else if (function_name_lowercase == "grouping")
return std::make_unique<FunctionLayer>(function_name_lowercase, allow_function_parameters_);
else
return std::make_unique<FunctionLayer>(function_name, allow_function_parameters_);
return std::make_unique<FunctionLayer>(function_name, allow_function_parameters_, identifier->as<ASTIdentifier>()->compound());
}
@ -2219,7 +2221,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr identifier;
if (ParserIdentifier(true).parse(pos, identifier, expected)
if (ParserCompoundIdentifier(false,true).parse(pos, identifier, expected)
&& ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
{
auto start = getFunctionLayer(identifier, is_table_function, allow_function_parameters);

View File

@ -0,0 +1,80 @@
#include <Parsers/FunctionParameterValuesVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/parseQuery.h>
#include <Common/FieldVisitorToString.h>
#include <Parsers/ASTHelpers.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
class FunctionParameterValuesVisitor
{
public:
explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_)
: parameter_values(parameter_values_)
{
}
void visit(const ASTPtr & ast)
{
if (const auto * function = ast->as<ASTFunction>())
visitFunction(*function);
for (const auto & child : ast->children)
visit(child);
}
private:
NameToNameMap & parameter_values;
void visitFunction(const ASTFunction & parameter_function)
{
if (parameter_function.name != "equals" && parameter_function.children.size() != 1)
return;
const auto * expression_list = parameter_function.children[0]->as<ASTExpressionList>();
if (expression_list && expression_list->children.size() != 2)
return;
if (const auto * identifier = expression_list->children[0]->as<ASTIdentifier>())
{
if (const auto * literal = expression_list->children[1]->as<ASTLiteral>())
{
parameter_values[identifier->name()] = convertFieldToString(literal->value);
}
else if (const auto * function = expression_list->children[1]->as<ASTFunction>())
{
if (isFunctionCast(function))
{
const auto * cast_expression = assert_cast<ASTExpressionList*>(function->arguments.get());
if (cast_expression->children.size() != 2)
throw Exception("Function CAST must have exactly two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (const auto * cast_literal = cast_expression->children[0]->as<ASTLiteral>())
{
parameter_values[identifier->name()] = convertFieldToString(cast_literal->value);
}
}
}
}
}
};
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast)
{
NameToNameMap parameter_values;
FunctionParameterValuesVisitor(parameter_values).visit(ast);
return parameter_values;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Core/Names.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
/// Find parameters in a query parameter values and collect them into map.
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast);
}

View File

@ -0,0 +1,73 @@
#include <Parsers/QueryParameterVisitor.h>
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
namespace DB
{
class QueryParameterVisitor
{
public:
explicit QueryParameterVisitor(NameToNameMap & parameters)
: query_parameters(parameters)
{
}
void visit(const ASTPtr & ast)
{
if (const auto & query_parameter = ast->as<ASTQueryParameter>())
visitQueryParameter(*query_parameter);
else
{
for (const auto & child : ast->children)
visit(child);
}
}
private:
NameToNameMap & query_parameters;
void visitQueryParameter(const ASTQueryParameter & query_parameter)
{
query_parameters[query_parameter.name]= query_parameter.type;
}
};
NameSet analyzeReceiveQueryParams(const std::string & query)
{
NameToNameMap query_params;
const char * query_begin = query.data();
const char * query_end = query.data() + query.size();
ParserQuery parser(query_end);
ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0);
QueryParameterVisitor(query_params).visit(extract_query_ast);
NameSet query_param_names;
for (const auto & query_param : query_params)
query_param_names.insert(query_param.first);
return query_param_names;
}
NameSet analyzeReceiveQueryParams(const ASTPtr & ast)
{
NameToNameMap query_params;
QueryParameterVisitor(query_params).visit(ast);
NameSet query_param_names;
for (const auto & query_param : query_params)
query_param_names.insert(query_param.first);
return query_param_names;
}
NameToNameMap analyzeReceiveQueryParamsWithType(const ASTPtr & ast)
{
NameToNameMap query_params;
QueryParameterVisitor(query_params).visit(ast);
return query_params;
}
}

View File

@ -2,6 +2,7 @@
#include <string>
#include <Core/Names.h>
#include <Parsers/IAST_fwd.h>
namespace DB
@ -10,4 +11,8 @@ namespace DB
/// Find parameters in a query and collect them into set.
NameSet analyzeReceiveQueryParams(const std::string & query);
NameSet analyzeReceiveQueryParams(const ASTPtr & ast);
NameToNameMap analyzeReceiveQueryParamsWithType(const ASTPtr & ast);
}

View File

@ -0,0 +1,66 @@
#include <Planner/CollectColumnIdentifiers.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ColumnNode.h>
#include <Planner/PlannerContext.h>
namespace DB
{
namespace
{
class CollectTopLevelColumnIdentifiersVisitor : public InDepthQueryTreeVisitor<CollectTopLevelColumnIdentifiersVisitor, true>
{
public:
explicit CollectTopLevelColumnIdentifiersVisitor(const PlannerContextPtr & planner_context_, ColumnIdentifierSet & used_identifiers_)
: used_identifiers(used_identifiers_)
, planner_context(planner_context_)
{}
static bool needChildVisit(VisitQueryTreeNodeType &, VisitQueryTreeNodeType & child)
{
const auto & node_type = child->getNodeType();
return node_type != QueryTreeNodeType::TABLE
&& node_type != QueryTreeNodeType::TABLE_FUNCTION
&& node_type != QueryTreeNodeType::QUERY
&& node_type != QueryTreeNodeType::UNION
&& node_type != QueryTreeNodeType::JOIN
&& node_type != QueryTreeNodeType::ARRAY_JOIN;
}
void visitImpl(const QueryTreeNodePtr & node)
{
if (node->getNodeType() != QueryTreeNodeType::COLUMN)
return;
const auto * column_identifier = planner_context->getColumnNodeIdentifierOrNull(node);
if (!column_identifier)
return;
used_identifiers.insert(*column_identifier);
}
ColumnIdentifierSet & used_identifiers;
const PlannerContextPtr & planner_context;
};
}
void collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out)
{
CollectTopLevelColumnIdentifiersVisitor visitor(planner_context, out);
visitor.visit(node);
}
ColumnIdentifierSet collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context)
{
ColumnIdentifierSet out;
collectTopLevelColumnIdentifiers(node, planner_context, out);
return out;
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Planner/Planner.h>
#include <Planner/TableExpressionData.h>
namespace DB
{
/** Collect all top level column identifiers from query tree node.
* Top level column identifiers are in the SELECT list or GROUP BY/ORDER BY/WHERE/HAVING clause, but not in child nodes of join tree.
* For example, in the following query:
* SELECT sum(b) FROM (SELECT x AS a, y AS b FROM t) AS t1 JOIN t2 ON t1.a = t2.key GROUP BY t2.y
* The top level column identifiers are: `t1.b`, `t2.y`
*
* There is precondition that table expression data is collected in planner context.
*/
ColumnIdentifierSet collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context);
void collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out);
}

View File

@ -64,6 +64,7 @@
#include <Planner/CollectTableExpressionData.h>
#include <Planner/PlannerJoinTree.h>
#include <Planner/PlannerExpressionAnalysis.h>
#include <Planner/CollectColumnIdentifiers.h>
namespace DB
{
@ -374,7 +375,9 @@ void Planner::buildQueryPlanIfNeeded()
collectSets(query_tree, *planner_context);
query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context);
auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context);
query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, top_level_identifiers, planner_context);
auto expression_analysis_result = buildExpressionAnalysisResult(query_tree, query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), planner_context);
if (expression_analysis_result.hasWhere())

View File

@ -33,6 +33,7 @@
#include <Interpreters/HashJoin.h>
#include <Interpreters/ArrayJoinAction.h>
#include <Planner/CollectColumnIdentifiers.h>
#include <Planner/Planner.h>
#include <Planner/PlannerJoins.h>
#include <Planner/PlannerActionsVisitor.h>
@ -262,19 +263,25 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context)
{
auto & join_node = join_tree_node->as<JoinNode &>();
ColumnIdentifierSet current_scope_columns = outer_scope_columns;
collectTopLevelColumnIdentifiers(join_tree_node, planner_context, current_scope_columns);
auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(),
select_query_info,
select_query_options,
current_scope_columns,
planner_context);
auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(),
select_query_info,
select_query_options,
current_scope_columns,
planner_context);
auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
@ -641,6 +648,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
size_t max_block_size = query_context->getSettingsRef().max_block_size;
size_t max_streams = query_context->getSettingsRef().max_threads;
JoinPipelineType join_pipeline_type = join_algorithm->pipelineType();
auto join_step = std::make_unique<JoinStep>(
left_plan.getCurrentDataStream(),
right_plan.getCurrentDataStream(),
@ -649,7 +657,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
max_streams,
false /*optimize_read_in_order*/);
join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst));
join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type));
std::vector<QueryPlanPtr> plans;
plans.emplace_back(std::make_unique<QueryPlan>(std::move(left_plan)));
@ -664,8 +672,13 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
for (auto & output : drop_unused_columns_after_join_actions_dag->getOutputs())
{
if (updated_outputs_names.contains(output->result_name) || !planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name))
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
if (updated_outputs_names.contains(output->result_name)
|| !global_planner_context->hasColumnIdentifier(output->result_name)
|| !outer_scope_columns.contains(output->result_name))
{
continue;
}
updated_outputs.push_back(output);
updated_outputs_names.insert(output->result_name);
@ -683,6 +696,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context)
{
auto & array_join_node = table_expression->as<ArrayJoinNode &>();
@ -690,6 +704,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
auto plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(),
select_query_info,
select_query_options,
outer_scope_columns,
planner_context);
auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
@ -729,6 +744,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node,
SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context)
{
auto join_tree_node_type = join_tree_node->getNodeType();
@ -747,11 +763,11 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node,
}
case QueryTreeNodeType::JOIN:
{
return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context);
return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, outer_scope_columns, planner_context);
}
case QueryTreeNodeType::ARRAY_JOIN:
{
return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context);
return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, outer_scope_columns, planner_context);
}
default:
{

View File

@ -15,6 +15,7 @@ namespace DB
QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node,
SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context);
}

View File

@ -13,6 +13,7 @@ namespace ErrorCodes
using ColumnIdentifier = std::string;
using ColumnIdentifiers = std::vector<ColumnIdentifier>;
using ColumnIdentifierSet = std::unordered_set<ColumnIdentifier>;
/** Table expression data is created for each table expression that take part in query.
* Table expression data has information about columns that participate in query, their name to identifier mapping,

View File

@ -15,7 +15,7 @@
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryParameterVisitor.h>
#include <Parsers/QueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/Session.h>
#include <Server/HTTPHandlerFactory.h>

View File

@ -3780,7 +3780,7 @@ std::pair<size_t, size_t> MergeTreeData::getMaxPartsCountAndSizeForPartition() c
}
size_t MergeTreeData::getMaxInactivePartsCountForPartition() const
size_t MergeTreeData::getMaxOutdatedPartsCountForPartition() const
{
return getMaxPartsCountAndSizeForPartitionWithState(DataPartState::Outdated).first;
}
@ -3801,70 +3801,102 @@ std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
}
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, ContextPtr query_context) const
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const
{
const auto settings = getSettings();
const auto & query_settings = query_context->getSettingsRef();
const size_t parts_count_in_total = getPartsCount();
/// check if have too many parts in total
if (parts_count_in_total >= settings->max_parts_in_total)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS);
throw Exception(
ErrorCodes::TOO_MANY_PARTS,
"Too many parts ({}) in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified "
"with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting.",
toString(parts_count_in_total));
}
auto [parts_count_in_partition, size_of_partition] = getMaxPartsCountAndSizeForPartition();
ssize_t k_inactive = -1;
if (settings->inactive_parts_to_throw_insert > 0 || settings->inactive_parts_to_delay_insert > 0)
size_t outdated_parts_over_threshold = 0;
{
size_t inactive_parts_count_in_partition = getMaxInactivePartsCountForPartition();
if (settings->inactive_parts_to_throw_insert > 0 && inactive_parts_count_in_partition >= settings->inactive_parts_to_throw_insert)
size_t outdated_parts_count_in_partition = 0;
if (settings->inactive_parts_to_throw_insert > 0 || settings->inactive_parts_to_delay_insert > 0)
outdated_parts_count_in_partition = getMaxOutdatedPartsCountForPartition();
if (settings->inactive_parts_to_throw_insert > 0 && outdated_parts_count_in_partition >= settings->inactive_parts_to_throw_insert)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception(
ErrorCodes::TOO_MANY_PARTS,
"Too many inactive parts ({}). Parts cleaning are processing significantly slower than inserts",
inactive_parts_count_in_partition);
outdated_parts_count_in_partition);
}
k_inactive = static_cast<ssize_t>(inactive_parts_count_in_partition) - static_cast<ssize_t>(settings->inactive_parts_to_delay_insert);
if (settings->inactive_parts_to_delay_insert > 0 && outdated_parts_count_in_partition >= settings->inactive_parts_to_delay_insert)
outdated_parts_over_threshold = outdated_parts_count_in_partition - settings->inactive_parts_to_delay_insert + 1;
}
auto parts_to_delay_insert = query_settings.parts_to_delay_insert ? query_settings.parts_to_delay_insert : settings->parts_to_delay_insert;
auto parts_to_throw_insert = query_settings.parts_to_throw_insert ? query_settings.parts_to_throw_insert : settings->parts_to_throw_insert;
auto [parts_count_in_partition, size_of_partition] = getMaxPartsCountAndSizeForPartition();
size_t average_part_size = parts_count_in_partition ? size_of_partition / parts_count_in_partition : 0;
bool parts_are_large_enough_in_average = settings->max_avg_part_size_for_too_many_parts
&& average_part_size > settings->max_avg_part_size_for_too_many_parts;
if (parts_count_in_partition >= parts_to_throw_insert && !parts_are_large_enough_in_average)
const auto active_parts_to_delay_insert
= query_settings.parts_to_delay_insert ? query_settings.parts_to_delay_insert : settings->parts_to_delay_insert;
const auto active_parts_to_throw_insert
= query_settings.parts_to_throw_insert ? query_settings.parts_to_throw_insert : settings->parts_to_throw_insert;
size_t active_parts_over_threshold = 0;
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception(
ErrorCodes::TOO_MANY_PARTS,
"Too many parts ({} with average size of {}). Merges are processing significantly slower than inserts",
parts_count_in_partition, ReadableSize(average_part_size));
bool parts_are_large_enough_in_average
= settings->max_avg_part_size_for_too_many_parts && average_part_size > settings->max_avg_part_size_for_too_many_parts;
if (parts_count_in_partition >= active_parts_to_throw_insert && !parts_are_large_enough_in_average)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception(
ErrorCodes::TOO_MANY_PARTS,
"Too many parts ({} with average size of {}). Merges are processing significantly slower than inserts",
parts_count_in_partition,
ReadableSize(average_part_size));
}
if (active_parts_to_delay_insert > 0 && parts_count_in_partition >= active_parts_to_delay_insert
&& !parts_are_large_enough_in_average)
/// if parts_count == parts_to_delay_insert -> we're 1 part over threshold
active_parts_over_threshold = parts_count_in_partition - active_parts_to_delay_insert + 1;
}
if (k_inactive < 0 && (parts_count_in_partition < parts_to_delay_insert || parts_are_large_enough_in_average))
/// no need for delay
if (!active_parts_over_threshold && !outdated_parts_over_threshold)
return;
const ssize_t k_active = ssize_t(parts_count_in_partition) - ssize_t(parts_to_delay_insert);
size_t max_k;
size_t k;
if (k_active > k_inactive)
UInt64 delay_milliseconds = 0;
{
max_k = parts_to_throw_insert - parts_to_delay_insert;
k = k_active + 1;
}
else
{
max_k = settings->inactive_parts_to_throw_insert - settings->inactive_parts_to_delay_insert;
k = k_inactive + 1;
}
size_t parts_over_threshold = 0;
size_t allowed_parts_over_threshold = 1;
const bool use_active_parts_threshold = (active_parts_over_threshold >= outdated_parts_over_threshold);
if (use_active_parts_threshold)
{
parts_over_threshold = active_parts_over_threshold;
allowed_parts_over_threshold = active_parts_to_throw_insert - active_parts_to_delay_insert;
}
else
{
parts_over_threshold = outdated_parts_over_threshold;
allowed_parts_over_threshold = outdated_parts_over_threshold; /// if throw threshold is not set, will use max delay
if (settings->inactive_parts_to_throw_insert > 0)
allowed_parts_over_threshold = settings->inactive_parts_to_throw_insert - settings->inactive_parts_to_delay_insert;
}
const UInt64 max_delay_milliseconds = (settings->max_delay_to_insert > 0 ? settings->max_delay_to_insert * 1000 : 1000);
/// min() as a save guard here
const UInt64 delay_milliseconds
= std::min(max_delay_milliseconds, static_cast<UInt64>(::pow(max_delay_milliseconds, static_cast<double>(k) / max_k)));
if (allowed_parts_over_threshold == 0 || parts_over_threshold > allowed_parts_over_threshold) [[unlikely]]
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Incorrect calculation of {} parts over threshold: allowed_parts_over_threshold={}, parts_over_threshold={}",
(use_active_parts_threshold ? "active" : "inactive"),
allowed_parts_over_threshold,
parts_over_threshold);
const UInt64 max_delay_milliseconds = (settings->max_delay_to_insert > 0 ? settings->max_delay_to_insert * 1000 : 1000);
double delay_factor = static_cast<double>(parts_over_threshold) / allowed_parts_over_threshold;
const UInt64 min_delay_milliseconds = settings->min_delay_to_insert_ms;
delay_milliseconds = std::max(min_delay_milliseconds, static_cast<UInt64>(max_delay_milliseconds * delay_factor));
}
ProfileEvents::increment(ProfileEvents::DelayedInserts);
ProfileEvents::increment(ProfileEvents::DelayedInsertsMilliseconds, delay_milliseconds);

View File

@ -535,7 +535,7 @@ public:
std::pair<size_t, size_t> getMaxPartsCountAndSizeForPartitionWithState(DataPartState state) const;
std::pair<size_t, size_t> getMaxPartsCountAndSizeForPartition() const;
size_t getMaxInactivePartsCountForPartition() const;
size_t getMaxOutdatedPartsCountForPartition() const;
/// Get min value of part->info.getDataVersion() for all active parts.
/// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition.
@ -555,7 +555,7 @@ public:
/// If the table contains too many active parts, sleep for a while to give them time to merge.
/// If until is non-null, wake up from the sleep earlier if the event happened.
void delayInsertOrThrowIfNeeded(Poco::Event * until, ContextPtr query_context) const;
void delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const;
/// Renames temporary part to a permanent part and adds it to the parts set.
/// It is assumed that the part does not intersect with existing parts.

View File

@ -5,6 +5,7 @@
#include <Interpreters/Context.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeIPv4andIPv6.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnTuple.h>
#include <Common/SipHash.h>
@ -93,9 +94,7 @@ namespace
}
void operator() (const IPv6 & x) const
{
UInt8 type = Field::Types::IPv6;
hash.update(type);
hash.update(x);
return operator()(String(reinterpret_cast<const char *>(&x), 16));
}
void operator() (const Float64 & x) const
{
@ -213,7 +212,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
bool are_all_integral = true;
for (const Field & field : value)
{
if (field.getType() != Field::Types::UInt64 && field.getType() != Field::Types::Int64)
if (field.getType() != Field::Types::UInt64 && field.getType() != Field::Types::Int64 && field.getType() != Field::Types::IPv4)
{
are_all_integral = false;
break;
@ -232,6 +231,8 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
if (typeid_cast<const DataTypeDate *>(partition_key_sample.getByPosition(i).type.get()))
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>())));
else if (typeid_cast<const DataTypeIPv4 *>(partition_key_sample.getByPosition(i).type.get()))
result += toString(value[i].get<IPv4>().toUnderType());
else
result += applyVisitor(to_string_visitor, value[i]);

View File

@ -68,12 +68,13 @@ struct Settings;
M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \
\
/** Inserts settings. */ \
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \
M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \
M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \
M(UInt64, inactive_parts_to_throw_insert, 0, "If more than this number inactive parts in single partition, throw 'Too many inactive parts ...' exception.", 0) \
M(UInt64, max_avg_part_size_for_too_many_parts, 10ULL * 1024 * 1024 * 1024, "The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the relevant partition) is not larger than the specified threshold. If it is larger than the specified threshold, the INSERTs will be neither delayed or rejected. This allows to have hundreds of terabytes in a single table on a single server if the parts are successfully merged to larger parts. This does not affect the thresholds on inactive parts or total parts.", 0) \
M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \
M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \
M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \
\
/* Part removal settings. */ \

View File

@ -237,6 +237,8 @@ struct SelectQueryInfo
Block minmax_count_projection_block;
MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr;
bool is_parameterized_view = false;
// If limit is not 0, that means it's a trivial limit query.
UInt64 limit = 0;

View File

@ -92,8 +92,6 @@ public:
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
private:
void init();
StorageS3::S3Configuration base_configuration;
std::shared_ptr<StorageS3> s3engine;
Poco::Logger * log;

View File

@ -229,11 +229,13 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr<TableJoin> analyzed_join,
return join_clone;
}
void StorageJoin::insertBlock(const Block & block, ContextPtr context)
{
Block block_to_insert = block;
convertRightBlock(block_to_insert);
TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context);
join->addJoinedBlock(block, true);
join->addJoinedBlock(block_to_insert, true);
}
size_t StorageJoin::getSize(ContextPtr context) const
@ -265,6 +267,16 @@ ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & bl
return join->joinGet(block, block_with_columns_to_add);
}
void StorageJoin::convertRightBlock(Block & block) const
{
bool need_covert = use_nulls && isLeftOrFull(kind);
if (!need_covert)
return;
for (auto & col : block)
JoinCommon::convertColumnToNullable(col);
}
void registerStorageJoin(StorageFactory & factory)
{
auto creator_fn = [](const StorageFactory::Arguments & args)

View File

@ -77,9 +77,7 @@ public:
{
auto metadata_snapshot = getInMemoryMetadataPtr();
Block block = metadata_snapshot->getSampleBlock();
if (use_nulls && isLeftOrFull(kind))
for (auto & col : block)
JoinCommon::convertColumnToNullable(col);
convertRightBlock(block);
return block;
}
@ -108,6 +106,8 @@ private:
void finishInsert() override {}
size_t getSize(ContextPtr context) const override;
RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const;
void convertRightBlock(Block & block) const;
};
}

View File

@ -287,7 +287,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, restarting_thread(*this)
, part_moves_between_shards_orchestrator(*this)
, renaming_restrictions(renaming_restrictions_)
, replicated_fetches_pool_size(getContext()->getFetchesExecutor()->getMaxTasksCount())
, replicated_fetches_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler()))
, replicated_sends_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler()))
{
@ -3113,6 +3112,7 @@ bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEnt
return false;
}
auto replicated_fetches_pool_size = getContext()->getFetchesExecutor()->getMaxTasksCount();
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundFetchesPoolTask].load(std::memory_order_relaxed);
if (busy_threads_in_pool >= replicated_fetches_pool_size)
{

View File

@ -463,8 +463,6 @@ private:
/// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro
const RenamingRestrictions renaming_restrictions;
const size_t replicated_fetches_pool_size;
/// Throttlers used in DataPartsExchange to lower maximum fetch/sends
/// speed.
ThrottlerPtr replicated_fetches_throttler;

View File

@ -3,6 +3,7 @@
#include <Storages/IStorage.h>
#include <DataTypes/ObjectUtils.h>
#include <DataTypes/NestedUtils.h>
#include <Storages/StorageView.h>
#include <sparsehash/dense_hash_set>
namespace DB
@ -112,33 +113,41 @@ NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, co
return *column;
}
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values) const
{
Block res;
const auto & columns = getMetadataForQuery()->getColumns();
for (const auto & name : column_names)
for (const auto & column_name : column_names)
{
auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, name);
auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, name);
std::string substituted_column_name = column_name;
/// substituted_column_name is used for parameterized view (which are created using query parameters
/// and SELECT is used with substitution of these query parameters )
if (!parameter_values.empty())
substituted_column_name = StorageView::replaceValueWithQueryParameter(column_name, parameter_values);
auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name);
auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name);
if (column && !object_column)
{
res.insert({column->type->createColumn(), column->type, column->name});
res.insert({column->type->createColumn(), column->type, column_name});
}
else if (object_column)
{
res.insert({object_column->type->createColumn(), object_column->type, object_column->name});
res.insert({object_column->type->createColumn(), object_column->type, column_name});
}
else if (auto it = virtual_columns.find(name); it != virtual_columns.end())
else if (auto it = virtual_columns.find(column_name); it != virtual_columns.end())
{
/// Virtual columns must be appended after ordinary, because user can
/// override them.
const auto & type = it->second;
res.insert({type->createColumn(), type, name});
res.insert({type->createColumn(), type, column_name});
}
else
{
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
"Column {} not found in table {}", backQuote(name), storage.getStorageID().getNameForLogs());
"Column {} not found in table {}", backQuote(substituted_column_name), storage.getStorageID().getNameForLogs());
}
}
return res;

View File

@ -66,7 +66,7 @@ struct StorageSnapshot
NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const;
/// Block with ordinary + materialized + aliases + virtuals + subcolumns.
Block getSampleBlockForColumns(const Names & column_names) const;
Block getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values = {}) const;
ColumnsDescription getDescriptionForColumns(const Names & column_names) const;

View File

@ -24,6 +24,9 @@
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Parsers/QueryParameterVisitor.h>
namespace DB
{
@ -114,6 +117,8 @@ StorageView::StorageView(
SelectQueryDescription description;
description.inner_query = query.select->ptr();
is_parameterized_view = query.isParameterizedView();
parameter_types = analyzeReceiveQueryParamsWithType(description.inner_query);
storage_metadata.setSelectQuery(description);
setInMemoryMetadata(storage_metadata);
}
@ -162,7 +167,7 @@ void StorageView::read(
query_plan.addStep(std::move(materializing));
/// And also convert to expected structure.
const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names);
const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names,parameter_values);
const auto & header = query_plan.getCurrentDataStream().header;
const auto * select_with_union = current_inner_query->as<ASTSelectWithUnionQuery>();
@ -198,20 +203,30 @@ static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_quer
return select_element->table_expression->as<ASTTableExpression>();
}
void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name)
void StorageView::replaceQueryParametersIfParametrizedView(ASTPtr & outer_query)
{
ReplaceQueryParameterVisitor visitor(parameter_values);
visitor.visit(outer_query);
}
void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name, bool parameterized_view)
{
ASTTableExpression * table_expression = getFirstTableExpression(outer_query);
if (!table_expression->database_and_table_name)
{
// If it's a view or merge table function, add a fake db.table name.
/// If it's a view or merge table function, add a fake db.table name.
/// For parameterized view, the function name is the db.view name, so add the function name
if (table_expression->table_function)
{
auto table_function_name = table_expression->table_function->as<ASTFunction>()->name;
if (table_function_name == "view" || table_function_name == "viewIfPermitted")
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
if (table_function_name == "merge")
else if (table_function_name == "merge")
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__merge");
else if (parameterized_view)
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>(table_function_name);
}
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
@ -229,6 +244,47 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
for (auto & child : table_expression->children)
if (child.get() == view_name.get())
child = view_query;
else if (child.get()
&& child->as<ASTFunction>()
&& table_expression->table_function
&& table_expression->table_function->as<ASTFunction>()
&& child->as<ASTFunction>()->name == table_expression->table_function->as<ASTFunction>()->name)
child = view_query;
}
String StorageView::replaceQueryParameterWithValue(const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types)
{
std::string name = column_name;
std::string::size_type pos = 0u;
for (const auto & parameter : parameter_values)
{
if ((pos = name.find(parameter.first)) != std::string::npos)
{
auto parameter_datatype_iterator = parameter_types.find(parameter.first);
if (parameter_datatype_iterator != parameter_types.end())
{
String parameter_name("_CAST(" + parameter.second + ", '" + parameter_datatype_iterator->second + "')");
name.replace(pos, parameter.first.size(), parameter_name);
break;
}
}
}
return name;
}
String StorageView::replaceValueWithQueryParameter(const String & column_name, const NameToNameMap & parameter_values)
{
String name = column_name;
std::string::size_type pos = 0u;
for (const auto & parameter : parameter_values)
{
if ((pos = name.find("_CAST(" + parameter.second)) != std::string::npos)
{
name = name.substr(0,pos) + parameter.first + ")";
break;
}
}
return name;
}
ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name)

View File

@ -19,6 +19,7 @@ public:
std::string getName() const override { return "View"; }
bool isView() const override { return true; }
bool isParameterizedView() const { return is_parameterized_view; }
/// It is passed inside the query and solved at its level.
bool supportsSampling() const override { return true; }
@ -34,13 +35,32 @@ public:
size_t max_block_size,
size_t num_streams) override;
static void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot)
void replaceQueryParametersIfParametrizedView(ASTPtr & outer_query);
static void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot, const bool parameterized_view)
{
replaceWithSubquery(select_query, metadata_snapshot->getSelectQuery().inner_query->clone(), view_name);
replaceWithSubquery(select_query, metadata_snapshot->getSelectQuery().inner_query->clone(), view_name, parameterized_view);
}
static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name);
static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name, const bool parameterized_view);
static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name);
static String replaceQueryParameterWithValue (const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types);
static String replaceValueWithQueryParameter (const String & column_name, const NameToNameMap & parameter_values);
void setParameterValues (NameToNameMap parameter_values_)
{
parameter_values = parameter_values_;
}
NameToNameMap getParameterValues() const
{
return parameter_types;
}
protected:
bool is_parameterized_view;
NameToNameMap parameter_values;
NameToNameMap parameter_types;
};
}

View File

@ -60,7 +60,7 @@ static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from
}
Pipe StorageSystemDetachedParts::read(
const Names & /* column_names */,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
@ -68,37 +68,63 @@ Pipe StorageSystemDetachedParts::read(
const size_t /*max_block_size*/,
const size_t /*num_streams*/)
{
storage_snapshot->check(column_names);
StoragesInfoStream stream(query_info, context);
/// Create the result.
Block block = storage_snapshot->metadata->getSampleBlock();
MutableColumns new_columns = block.cloneEmptyColumns();
NameSet names_set(column_names.begin(), column_names.end());
std::vector<UInt8> columns_mask(block.columns());
Block header;
for (size_t i = 0; i < block.columns(); ++i)
{
if (names_set.contains(block.getByPosition(i).name))
{
columns_mask[i] = 1;
header.insert(block.getByPosition(i));
}
}
MutableColumns new_columns = header.cloneEmptyColumns();
while (StoragesInfo info = stream.next())
{
const auto parts = info.data->getDetachedParts();
for (const auto & p : parts)
{
size_t i = 0;
size_t src_index = 0, res_index = 0;
String detached_part_path = fs::path(MergeTreeData::DETACHED_DIR_NAME) / p.dir_name;
new_columns[i++]->insert(info.database);
new_columns[i++]->insert(info.table);
new_columns[i++]->insert(p.valid_name ? p.partition_id : Field());
new_columns[i++]->insert(p.dir_name);
new_columns[i++]->insert(calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path));
new_columns[i++]->insert(p.disk->getName());
new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string());
new_columns[i++]->insert(p.valid_name ? p.prefix : Field());
new_columns[i++]->insert(p.valid_name ? p.min_block : Field());
new_columns[i++]->insert(p.valid_name ? p.max_block : Field());
new_columns[i++]->insert(p.valid_name ? p.level : Field());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(info.database);
if (columns_mask[src_index++])
new_columns[res_index++]->insert(info.table);
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.valid_name ? p.partition_id : Field());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.dir_name);
if (columns_mask[src_index++])
new_columns[res_index++]->insert(calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path));
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.disk->getName());
if (columns_mask[src_index++])
new_columns[res_index++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.valid_name ? p.prefix : Field());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.valid_name ? p.min_block : Field());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.valid_name ? p.max_block : Field());
if (columns_mask[src_index++])
new_columns[res_index++]->insert(p.valid_name ? p.level : Field());
}
}
UInt64 num_rows = new_columns.at(0)->size();
Chunk chunk(std::move(new_columns), num_rows);
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk)));
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(header), std::move(chunk)));
}
}

View File

@ -39,21 +39,20 @@ DB::StoragePtr createStorage(DB::DiskPtr & disk)
return table;
}
template <typename T>
class StorageLogTest : public testing::Test
{
public:
void SetUp() override
{
disk = createDisk<T>();
disk = createDisk();
table = createStorage(disk);
}
void TearDown() override
{
table->flushAndShutdown();
destroyDisk<T>(disk);
destroyDisk(disk);
}
const DB::DiskPtr & getDisk() { return disk; }
@ -65,9 +64,6 @@ private:
};
using DiskImplementations = testing::Types<DB::DiskLocal>;
TYPED_TEST_SUITE(StorageLogTest, DiskImplementations);
// Returns data written to table in Values format.
std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr context)
{
@ -153,7 +149,7 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context)
return out_buf.str();
}
TYPED_TEST(StorageLogTest, testReadWrite)
TEST_F(StorageLogTest, testReadWrite)
{
using namespace DB;
const auto & context_holder = getContext();

View File

@ -7,6 +7,10 @@ import sys
from github import Github
from build_download_helper import get_build_name_for_check, read_build_urls
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from commit_status_helper import post_commit_status
from docker_pull_helper import get_image_with_version
from env_helper import (
GITHUB_REPOSITORY,
GITHUB_RUN_URL,
@ -14,15 +18,12 @@ from env_helper import (
REPO_COPY,
TEMP_PATH,
)
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from build_download_helper import get_build_name_for_check, read_build_urls
from docker_pull_helper import get_image_with_version
from commit_status_helper import post_commit_status
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from stopwatch import Stopwatch
from report import TestResult
from rerun_helper import RerunHelper
from s3_helper import S3Helper
from stopwatch import Stopwatch
IMAGE_NAME = "clickhouse/fuzzer"
@ -149,16 +150,15 @@ if __name__ == "__main__":
status = "failure"
description = "Task failed: $?=" + str(retcode)
test_result = TestResult(description, "OK")
if "fail" in status:
test_result = [(description, "FAIL")]
else:
test_result = [(description, "OK")]
test_result.status = "FAIL"
ch_helper = ClickHouseHelper()
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,
test_result,
[test_result],
status,
stopwatch.duration_seconds,
stopwatch.start_time_str,

View File

@ -1,18 +1,19 @@
#!/usr/bin/env python3
from typing import List, Tuple
import argparse
import csv
import itertools
import logging
import os
from github import Github
from s3_helper import S3Helper
from commit_status_helper import post_commit_status
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResults, TestResult
from s3_helper import S3Helper
from upload_result_helper import upload_results
from commit_status_helper import post_commit_status
def parse_args():
@ -21,11 +22,9 @@ def parse_args():
return parser.parse_args()
def post_commit_status_from_file(file_path):
res = []
def post_commit_status_from_file(file_path: str) -> List[str]:
with open(file_path, "r", encoding="utf-8") as f:
fin = csv.reader(f, delimiter="\t")
res = list(itertools.islice(fin, 1))
res = list(csv.reader(f, delimiter="\t"))
if len(res) < 1:
raise Exception(f'Can\'t read from "{file_path}"')
if len(res[0]) != 3:
@ -33,22 +32,22 @@ def post_commit_status_from_file(file_path):
return res[0]
def process_result(file_path):
test_results = []
def process_result(file_path: str) -> Tuple[bool, TestResults]:
test_results = [] # type: TestResults
state, report_url, description = post_commit_status_from_file(file_path)
prefix = os.path.basename(os.path.dirname(file_path))
is_ok = state == "success"
if is_ok and report_url == "null":
return is_ok, None
return is_ok, test_results
status = f'OK: Bug reproduced (<a href="{report_url}">Report</a>)'
if not is_ok:
status = f'Bug is not reproduced (<a href="{report_url}">Report</a>)'
test_results.append([f"{prefix}: {description}", status])
test_results.append(TestResult(f"{prefix}: {description}", status))
return is_ok, test_results
def process_all_results(file_paths):
def process_all_results(file_paths: str) -> Tuple[bool, TestResults]:
any_ok = False
all_results = []
for status_path in file_paths:

View File

@ -10,13 +10,14 @@ from typing import Dict, List, Tuple
from github import Github
from env_helper import (
GITHUB_JOB_URL,
GITHUB_REPOSITORY,
GITHUB_RUN_URL,
GITHUB_SERVER_URL,
REPORTS_PATH,
TEMP_PATH,
)
from report import create_build_html_report
from report import create_build_html_report, BuildResult, BuildResults
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import NeedsDataType, PRInfo
@ -31,24 +32,6 @@ from rerun_helper import RerunHelper
NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "")
class BuildResult:
def __init__(
self,
compiler,
build_type,
sanitizer,
status,
elapsed_seconds,
with_coverage,
):
self.compiler = compiler
self.build_type = build_type
self.sanitizer = sanitizer
self.status = status
self.elapsed_seconds = elapsed_seconds
self.with_coverage = with_coverage
def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]:
groups = {
"apk": [],
@ -81,7 +64,7 @@ def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]:
def get_failed_report(
job_name: str,
) -> Tuple[List[BuildResult], List[List[str]], List[str]]:
) -> Tuple[BuildResults, List[List[str]], List[str]]:
message = f"{job_name} failed"
build_result = BuildResult(
compiler="unknown",
@ -89,14 +72,13 @@ def get_failed_report(
sanitizer="unknown",
status=message,
elapsed_seconds=0,
with_coverage=False,
)
return [build_result], [[""]], [GITHUB_RUN_URL]
def process_report(
build_report: dict,
) -> Tuple[List[BuildResult], List[List[str]], List[str]]:
) -> Tuple[BuildResults, List[List[str]], List[str]]:
build_config = build_report["build_config"]
build_result = BuildResult(
compiler=build_config["compiler"],
@ -104,7 +86,6 @@ def process_report(
sanitizer=build_config["sanitizer"],
status="success" if build_report["status"] else "failure",
elapsed_seconds=build_report["elapsed_seconds"],
with_coverage=False,
)
build_results = []
build_urls = []
@ -207,9 +188,9 @@ def main():
logging.info("Got exactly %s builds", len(builds_report_map))
# Group build artifacts by groups
build_results = [] # type: List[BuildResult]
build_artifacts = [] #
build_logs = []
build_results = [] # type: BuildResults
build_artifacts = [] # type: List[List[str]]
build_logs = [] # type: List[str]
for build_report in build_reports:
_build_results, build_artifacts_url, build_logs_url = process_report(
@ -244,7 +225,7 @@ def main():
branch_name = f"PR #{pr_info.number}"
branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/pull/{pr_info.number}"
commit_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commit/{pr_info.sha}"
task_url = GITHUB_RUN_URL
task_url = GITHUB_JOB_URL()
report = create_build_html_report(
build_check_name,
build_results,

View File

@ -1,10 +1,14 @@
#!/usr/bin/env python3
import time
import logging
from typing import List
import json
import logging
import time
import requests # type: ignore
from get_robot_token import get_parameter_from_ssm
from pr_info import PRInfo
from report import TestResults
class InsertException(Exception):
@ -129,14 +133,14 @@ class ClickHouseHelper:
def prepare_tests_results_for_clickhouse(
pr_info,
test_results,
check_status,
check_duration,
check_start_time,
report_url,
check_name,
):
pr_info: PRInfo,
test_results: TestResults,
check_status: str,
check_duration: float,
check_start_time: str,
report_url: str,
check_name: str,
) -> List[dict]:
pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master"
base_ref = "master"
@ -172,13 +176,11 @@ def prepare_tests_results_for_clickhouse(
result = [common_properties]
for test_result in test_results:
current_row = common_properties.copy()
test_name = test_result[0]
test_status = test_result[1]
test_name = test_result.name
test_status = test_result.status
test_time = 0
if len(test_result) > 2 and test_result[2]:
test_time = test_result[2]
current_row["test_duration_ms"] = int(float(test_time) * 1000)
test_time = test_result.time or 0
current_row["test_duration_ms"] = int(test_time * 1000)
current_row["test_name"] = test_name
current_row["test_status"] = test_status
result.append(current_row)
@ -186,7 +188,9 @@ def prepare_tests_results_for_clickhouse(
return result
def mark_flaky_tests(clickhouse_helper, check_name, test_results):
def mark_flaky_tests(
clickhouse_helper: ClickHouseHelper, check_name: str, test_results: TestResults
) -> None:
try:
query = f"""SELECT DISTINCT test_name
FROM checks
@ -202,7 +206,7 @@ WHERE
logging.info("Found flaky tests: %s", ", ".join(master_failed_tests))
for test_result in test_results:
if test_result[1] == "FAIL" and test_result[0] in master_failed_tests:
test_result[1] = "FLAKY"
if test_result.status == "FAIL" and test_result.name in master_failed_tests:
test_result.status = "FLAKY"
except Exception as ex:
logging.error("Exception happened during flaky tests fetch %s", ex)

View File

@ -7,6 +7,8 @@ import logging
from github import Github
from commit_status_helper import post_commit_status
from docker_pull_helper import get_image_with_version
from env_helper import (
IMAGES_PATH,
REPO_COPY,
@ -14,10 +16,9 @@ from env_helper import (
S3_TEST_REPORTS_BUCKET,
TEMP_PATH,
)
from commit_status_helper import post_commit_status
from docker_pull_helper import get_image_with_version
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -80,9 +81,9 @@ if __name__ == "__main__":
"HTML report</a>"
)
test_results = [(index_html, "Look at the report")]
test_result = TestResult(index_html, "Look at the report")
report_url = upload_results(s3_helper, 0, pr_info.sha, test_results, [], NAME)
report_url = upload_results(s3_helper, 0, pr_info.sha, [test_result], [], NAME)
print(f"::notice ::Report url: {report_url}")

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
from distutils.version import StrictVersion
from typing import List, Tuple
import logging
import os
import subprocess
@ -8,21 +9,22 @@ import sys
from github import Github
from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from build_download_helper import download_builds_filter
from upload_result_helper import upload_results
from docker_pull_helper import get_images_with_versions
from commit_status_helper import post_commit_status
from clickhouse_helper import (
ClickHouseHelper,
mark_flaky_tests,
prepare_tests_results_for_clickhouse,
)
from stopwatch import Stopwatch
from commit_status_helper import post_commit_status
from docker_pull_helper import get_images_with_versions
from env_helper import TEMP_PATH, REPORTS_PATH
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResults, TestResult
from rerun_helper import RerunHelper
from s3_helper import S3Helper
from stopwatch import Stopwatch
from upload_result_helper import upload_results
IMAGE_UBUNTU = "clickhouse/test-old-ubuntu"
IMAGE_CENTOS = "clickhouse/test-old-centos"
@ -31,18 +33,18 @@ DOWNLOAD_RETRIES_COUNT = 5
CHECK_NAME = "Compatibility check"
def process_os_check(log_path):
def process_os_check(log_path: str) -> TestResult:
name = os.path.basename(log_path)
with open(log_path, "r") as log:
line = log.read().split("\n")[0].strip()
if line != "OK":
return (name, "FAIL")
return TestResult(name, "FAIL")
else:
return (name, "OK")
return TestResult(name, "OK")
def process_glibc_check(log_path):
bad_lines = []
def process_glibc_check(log_path: str) -> TestResults:
test_results = [] # type: TestResults
with open(log_path, "r") as log:
for line in log:
if line.strip():
@ -50,32 +52,36 @@ def process_glibc_check(log_path):
symbol_with_glibc = columns[-2] # sysconf@GLIBC_2.2.5
_, version = symbol_with_glibc.split("@GLIBC_")
if version == "PRIVATE":
bad_lines.append((symbol_with_glibc, "FAIL"))
test_results.append(TestResult(symbol_with_glibc, "FAIL"))
elif StrictVersion(version) > MAX_GLIBC_VERSION:
bad_lines.append((symbol_with_glibc, "FAIL"))
if not bad_lines:
bad_lines.append(("glibc check", "OK"))
return bad_lines
test_results.append(TestResult(symbol_with_glibc, "FAIL"))
if not test_results:
test_results.append(TestResult("glibc check", "OK"))
return test_results
def process_result(result_folder, server_log_folder):
summary = process_glibc_check(os.path.join(result_folder, "glibc.log"))
def process_result(
result_folder: str, server_log_folder: str
) -> Tuple[str, str, TestResults, List[str]]:
test_results = process_glibc_check(os.path.join(result_folder, "glibc.log"))
status = "success"
description = "Compatibility check passed"
if len(summary) > 1 or summary[0][1] != "OK":
if len(test_results) > 1 or test_results[0].status != "OK":
status = "failure"
description = "glibc check failed"
if status == "success":
for operating_system in ("ubuntu:12.04", "centos:5"):
result = process_os_check(os.path.join(result_folder, operating_system))
if result[1] != "OK":
test_result = process_os_check(
os.path.join(result_folder, operating_system)
)
if test_result.status != "OK":
status = "failure"
description = f"Old {operating_system} failed"
summary += [result]
test_results += [test_result]
break
summary += [result]
test_results += [test_result]
server_log_path = os.path.join(server_log_folder, "clickhouse-server.log")
stderr_log_path = os.path.join(server_log_folder, "stderr.log")
@ -90,7 +96,7 @@ def process_result(result_folder, server_log_folder):
if os.path.exists(client_stderr_log_path):
result_logs.append(client_stderr_log_path)
return status, description, summary, result_logs
return status, description, test_results, result_logs
def get_run_commands(
@ -109,13 +115,12 @@ def get_run_commands(
]
if __name__ == "__main__":
def main():
logging.basicConfig(level=logging.INFO)
stopwatch = Stopwatch()
temp_path = TEMP_PATH
repo_path = REPO_COPY
reports_path = REPORTS_PATH
pr_info = PRInfo()
@ -201,5 +206,9 @@ if __name__ == "__main__":
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if state == "error":
if state == "failure":
sys.exit(1)
if __name__ == "__main__":
main()

View File

@ -8,6 +8,7 @@ import shutil
import subprocess
import time
import sys
from pathlib import Path
from typing import Any, Dict, List, Optional, Set, Tuple, Union
from github import Github
@ -17,6 +18,7 @@ from commit_status_helper import post_commit_status
from env_helper import GITHUB_WORKSPACE, RUNNER_TEMP, GITHUB_RUN_URL
from get_robot_token import get_best_robot_token, get_parameter_from_ssm
from pr_info import PRInfo
from report import TestResults, TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from upload_result_helper import upload_results
@ -182,11 +184,12 @@ def build_and_push_dummy_image(
image: DockerImage,
version_string: str,
push: bool,
) -> Tuple[bool, str]:
) -> Tuple[bool, Path]:
dummy_source = "ubuntu:20.04"
logging.info("Building docker image %s as %s", image.repo, dummy_source)
build_log = os.path.join(
TEMP_PATH, f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}"
build_log = (
Path(TEMP_PATH)
/ f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}.log"
)
with open(build_log, "wb") as bl:
cmd = (
@ -213,7 +216,7 @@ def build_and_push_one_image(
additional_cache: str,
push: bool,
child: bool,
) -> Tuple[bool, str]:
) -> Tuple[bool, Path]:
if image.only_amd64 and platform.machine() not in ["amd64", "x86_64"]:
return build_and_push_dummy_image(image, version_string, push)
logging.info(
@ -222,8 +225,9 @@ def build_and_push_one_image(
version_string,
image.full_path,
)
build_log = os.path.join(
TEMP_PATH, f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}"
build_log = (
Path(TEMP_PATH)
/ f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}.log"
)
push_arg = ""
if push:
@ -273,27 +277,42 @@ def process_single_image(
additional_cache: str,
push: bool,
child: bool,
) -> List[Tuple[str, str, str]]:
) -> TestResults:
logging.info("Image will be pushed with versions %s", ", ".join(versions))
result = []
results = [] # type: TestResults
for ver in versions:
stopwatch = Stopwatch()
for i in range(5):
success, build_log = build_and_push_one_image(
image, ver, additional_cache, push, child
)
if success:
result.append((image.repo + ":" + ver, build_log, "OK"))
results.append(
TestResult(
image.repo + ":" + ver,
"OK",
stopwatch.duration_seconds,
[build_log],
)
)
break
logging.info(
"Got error will retry %s time and sleep for %s seconds", i, i * 5
)
time.sleep(i * 5)
else:
result.append((image.repo + ":" + ver, build_log, "FAIL"))
results.append(
TestResult(
image.repo + ":" + ver,
"FAIL",
stopwatch.duration_seconds,
[build_log],
)
)
logging.info("Processing finished")
image.built = True
return result
return results
def process_image_with_parents(
@ -302,41 +321,19 @@ def process_image_with_parents(
additional_cache: str,
push: bool,
child: bool = False,
) -> List[Tuple[str, str, str]]:
result = [] # type: List[Tuple[str,str,str]]
) -> TestResults:
results = [] # type: TestResults
if image.built:
return result
return results
if image.parent is not None:
result += process_image_with_parents(
results += process_image_with_parents(
image.parent, versions, additional_cache, push, False
)
child = True
result += process_single_image(image, versions, additional_cache, push, child)
return result
def process_test_results(
s3_client: S3Helper, test_results: List[Tuple[str, str, str]], s3_path_prefix: str
) -> Tuple[str, List[Tuple[str, str]]]:
overall_status = "success"
processed_test_results = []
for image, build_log, status in test_results:
if status != "OK":
overall_status = "failure"
url_part = ""
if build_log is not None and os.path.exists(build_log):
build_url = s3_client.upload_test_report_to_s3(
build_log, s3_path_prefix + "/" + os.path.basename(build_log)
)
url_part += f'<a href="{build_url}">build_log</a>'
if url_part:
test_name = image + " (" + url_part + ")"
else:
test_name = image
processed_test_results.append((test_name, status))
return overall_status, processed_test_results
results += process_single_image(image, versions, additional_cache, push, child)
return results
def parse_args() -> argparse.Namespace:
@ -440,7 +437,7 @@ def main():
image_versions, result_version = gen_versions(pr_info, args.suffix)
result_images = {}
images_processing_result = []
test_results = [] # type: TestResults
additional_cache = ""
if pr_info.release_pr or pr_info.merged_pr:
additional_cache = str(pr_info.release_pr or pr_info.merged_pr)
@ -448,7 +445,7 @@ def main():
for image in changed_images:
# If we are in backport PR, then pr_info.release_pr is defined
# We use it as tag to reduce rebuilding time
images_processing_result += process_image_with_parents(
test_results += process_image_with_parents(
image, image_versions, additional_cache, args.push
)
result_images[image.repo] = result_version
@ -466,12 +463,9 @@ def main():
s3_helper = S3Helper()
s3_path_prefix = (
str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(" ", "_")
)
status, test_results = process_test_results(
s3_helper, images_processing_result, s3_path_prefix
)
status = "success"
if [r for r in test_results if r.status != "OK"]:
status = "failure"
url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME)
@ -495,7 +489,7 @@ def main():
ch_helper = ClickHouseHelper()
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if status == "error":
if status == "failure":
sys.exit(1)

Some files were not shown because too many files have changed in this diff Show More