Merge remote-tracking branch 'upstream/master' into rewrite-functions-to-subcolumns

This commit is contained in:
Anton Popov 2021-05-22 03:06:36 +03:00
commit e3ca1cf94d
126 changed files with 1497 additions and 1003 deletions

View File

@ -127,7 +127,7 @@
* Fixed some tests on AArch64 platform. [#22596](https://github.com/ClickHouse/ClickHouse/pull/22596) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Added function alignment for possibly better performance. [#21431](https://github.com/ClickHouse/ClickHouse/pull/21431) ([Danila Kutenin](https://github.com/danlark1)).
* Adjust some tests to output identical results on amd64 and aarch64 (qemu). The result was depending on implementation specific CPU behaviour. [#22590](https://github.com/ClickHouse/ClickHouse/pull/22590) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Allow query profiling only on x86_64. See [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174)#issuecomment-812954965 and [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638)#issuecomment-703805337. This closes [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638). [#22580](https://github.com/ClickHouse/ClickHouse/pull/22580) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Allow query profiling only on x86_64. See [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174#issuecomment-812954965) and [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638#issuecomment-703805337). This closes [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638). [#22580](https://github.com/ClickHouse/ClickHouse/pull/22580) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Allow building with unbundled xz (lzma) using `USE_INTERNAL_XZ_LIBRARY=OFF` CMake option. [#22571](https://github.com/ClickHouse/ClickHouse/pull/22571) ([Kfir Itzhak](https://github.com/mastertheknife)).
* Enable bundled `openldap` on `ppc64le` [#22487](https://github.com/ClickHouse/ClickHouse/pull/22487) ([Kfir Itzhak](https://github.com/mastertheknife)).
* Disable incompatible libraries (platform specific typically) on `ppc64le` [#22475](https://github.com/ClickHouse/ClickHouse/pull/22475) ([Kfir Itzhak](https://github.com/mastertheknife)).

View File

@ -78,6 +78,8 @@ PoolWithFailover::PoolWithFailover(
const RemoteDescription & addresses,
const std::string & user,
const std::string & password,
unsigned default_connections_,
unsigned max_connections_,
size_t max_tries_)
: max_tries(max_tries_)
, shareable(false)
@ -85,7 +87,13 @@ PoolWithFailover::PoolWithFailover(
/// Replicas have the same priority, but traversed replicas are moved to the end of the queue.
for (const auto & [host, port] : addresses)
{
replicas_by_priority[0].emplace_back(std::make_shared<Pool>(database, host, user, password, port));
replicas_by_priority[0].emplace_back(std::make_shared<Pool>(database,
host, user, password, port,
/* socket_ = */ "",
MYSQLXX_DEFAULT_TIMEOUT,
MYSQLXX_DEFAULT_RW_TIMEOUT,
default_connections_,
max_connections_));
}
}

View File

@ -115,6 +115,8 @@ namespace mysqlxx
const RemoteDescription & addresses,
const std::string & user,
const std::string & password,
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
PoolWithFailover(const PoolWithFailover & other);

View File

@ -15,7 +15,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
...
) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']);
) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause'])
SETTINGS
[connection_pool_size=16, ]
[connection_max_tries=3, ]
[connection_auto_close=true ]
;
```
See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query.

View File

@ -5,11 +5,11 @@ toc_title: "Функции для шифрования"
# Функции шифрования {#encryption-functions}
Даннвые функции реализуют шифрование и расшифровку данных с помощью AES (Advanced Encryption Standard) алгоритма.
Данные функции реализуют шифрование и расшифровку данных с помощью AES (Advanced Encryption Standard) алгоритма.
Длина ключа зависит от режима шифрования. Он может быть длинной в 16, 24 и 32 байта для режимов шифрования `-128-`, `-196-` и `-256-` соответственно.
Длина инициализирующего вектора всегда 16 байт (лишнии байты игнорируются).
Длина инициализирующего вектора всегда 16 байт (лишние байты игнорируются).
Обратите внимание, что до версии Clickhouse 21.1 эти функции работали медленно.

View File

@ -62,7 +62,6 @@ def build_amp(lang, args, cfg):
for root, _, filenames in os.walk(site_temp):
if 'index.html' in filenames:
paths.append(prepare_amp_html(lang, args, root, site_temp, main_site_dir))
test.test_amp(paths, lang)
logging.info(f'Finished building AMP version for {lang}')

View File

@ -40,7 +40,7 @@ def build_for_lang(lang, args):
site_names = {
'en': 'ClickHouse Blog',
'ru': 'Блог ClickHouse '
'ru': 'Блог ClickHouse'
}
assert len(site_names) == len(languages)
@ -62,7 +62,7 @@ def build_for_lang(lang, args):
strict=True,
theme=theme_cfg,
nav=blog_nav,
copyright='©20162020 Yandex LLC',
copyright='©20162021 Yandex LLC',
use_directory_urls=True,
repo_name='ClickHouse/ClickHouse',
repo_url='https://github.com/ClickHouse/ClickHouse/',

View File

@ -94,7 +94,7 @@ def build_for_lang(lang, args):
site_dir=site_dir,
strict=True,
theme=theme_cfg,
copyright='©20162020 Yandex LLC',
copyright='©20162021 Yandex LLC',
use_directory_urls=True,
repo_name='ClickHouse/ClickHouse',
repo_url='https://github.com/ClickHouse/ClickHouse/',

View File

@ -31,7 +31,16 @@ def build_nav_entry(root, args):
result_items.append((prio, title, payload))
elif filename.endswith('.md'):
path = os.path.join(root, filename)
meta, content = util.read_md_file(path)
meta = ''
content = ''
try:
meta, content = util.read_md_file(path)
except:
print('Error in file: {}'.format(path))
raise
path = path.split('/', 2)[-1]
title = meta.get('toc_title', find_first_header(content))
if title:

View File

@ -3,34 +3,9 @@
import logging
import os
import sys
import bs4
import logging
import os
import subprocess
import bs4
def test_amp(paths, lang):
try:
# Get latest amp validator version
subprocess.check_call('amphtml-validator --help',
stdout=subprocess.DEVNULL,
stderr=subprocess.DEVNULL,
shell=True)
except subprocess.CalledProcessError:
subprocess.check_call('npm i -g amphtml-validator', stderr=subprocess.DEVNULL, shell=True)
paths = ' '.join(paths)
command = f'amphtml-validator {paths}'
try:
subprocess.check_output(command, shell=True).decode('utf-8')
except subprocess.CalledProcessError:
logging.error(f'Invalid AMP for {lang}')
raise
def test_template(template_path):
if template_path.endswith('amp.html'):

View File

@ -1366,6 +1366,27 @@ private:
{
const auto * exception = server_exception ? server_exception.get() : client_exception.get();
fmt::print(stderr, "Error on processing query '{}': {}\n", ast_to_process->formatForErrorMessage(), exception->message());
// Try to reconnect after errors, for two reasons:
// 1. We might not have realized that the server died, e.g. if
// it sent us a <Fatal> trace and closed connection properly.
// 2. The connection might have gotten into a wrong state and
// the next query will get false positive about
// "Unknown packet from server".
try
{
connection->forceConnected(connection_parameters.timeouts);
}
catch (...)
{
// Just report it, we'll terminate below.
fmt::print(stderr,
"Error while reconnecting to the server: Code: {}: {}\n",
getCurrentExceptionCode(),
getCurrentExceptionMessage(true));
assert(!connection->isConnected());
}
}
if (!connection->isConnected())
@ -1469,11 +1490,6 @@ private:
server_exception.reset();
client_exception.reset();
have_error = false;
// We have to reinitialize connection after errors, because it
// might have gotten into a wrong state and we'll get false
// positives about "Unknown packet from server".
connection->forceConnected(connection_parameters.timeouts);
}
else if (ast_to_process->formatForErrorMessage().size() > 500)
{

View File

@ -52,6 +52,9 @@ template <typename Value, bool float_return> using FuncQuantilesTDigest = Aggreg
template <typename Value, bool float_return> using FuncQuantileTDigestWeighted = AggregateFunctionQuantile<Value, QuantileTDigest<Value>, NameQuantileTDigestWeighted, true, std::conditional_t<float_return, Float32, void>, false>;
template <typename Value, bool float_return> using FuncQuantilesTDigestWeighted = AggregateFunctionQuantile<Value, QuantileTDigest<Value>, NameQuantilesTDigestWeighted, true, std::conditional_t<float_return, Float32, void>, true>;
template <typename Value, bool float_return> using FuncQuantileBFloat16 = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantileBFloat16, false, std::conditional_t<float_return, Float64, void>, false>;
template <typename Value, bool float_return> using FuncQuantilesBFloat16 = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantilesBFloat16, false, std::conditional_t<float_return, Float64, void>, true>;
template <template <typename, bool> class Function>
static constexpr bool supportDecimal()
@ -156,6 +159,9 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
factory.registerFunction(NameQuantileTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantileTDigestWeighted>);
factory.registerFunction(NameQuantilesTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantilesTDigestWeighted>);
factory.registerFunction(NameQuantileBFloat16::name, createAggregateFunctionQuantile<FuncQuantileBFloat16>);
factory.registerFunction(NameQuantilesBFloat16::name, createAggregateFunctionQuantile<FuncQuantilesBFloat16>);
/// 'median' is an alias for 'quantile'
factory.registerAlias("median", NameQuantile::name);
factory.registerAlias("medianDeterministic", NameQuantileDeterministic::name);
@ -167,6 +173,7 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
factory.registerAlias("medianTimingWeighted", NameQuantileTimingWeighted::name);
factory.registerAlias("medianTDigest", NameQuantileTDigest::name);
factory.registerAlias("medianTDigestWeighted", NameQuantileTDigestWeighted::name);
factory.registerAlias("medianBFloat16", NameQuantileBFloat16::name);
}
}

View File

@ -9,6 +9,7 @@
#include <AggregateFunctions/QuantileExactWeighted.h>
#include <AggregateFunctions/QuantileTiming.h>
#include <AggregateFunctions/QuantileTDigest.h>
#include <AggregateFunctions/QuantileBFloat16Histogram.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/QuantilesCommon.h>
@ -228,4 +229,7 @@ struct NameQuantileTDigestWeighted { static constexpr auto name = "quantileTDige
struct NameQuantilesTDigest { static constexpr auto name = "quantilesTDigest"; };
struct NameQuantilesTDigestWeighted { static constexpr auto name = "quantilesTDigestWeighted"; };
struct NameQuantileBFloat16 { static constexpr auto name = "quantileBFloat16"; };
struct NameQuantilesBFloat16 { static constexpr auto name = "quantilesBFloat16"; };
}

View File

@ -0,0 +1,63 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSegmentLengthSum.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &)
{
if (arguments.size() != 2)
throw Exception(
"Aggregate function " + name + " requires two timestamps argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto args = {arguments[0].get(), arguments[1].get()};
if (WhichDataType{args.begin()[0]}.idx != WhichDataType{args.begin()[1]}.idx)
throw Exception(
"Illegal type " + args.begin()[0]->getName() + " and " + args.begin()[1]->getName() + " of arguments of aggregate function "
+ name + ", there two arguments should have same DataType",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto & arg : args)
{
if (!isNativeNumber(arg) && !isDateOrDateTime(arg))
throw Exception(
"Illegal type " + arg->getName() + " of argument of aggregate function " + name
+ ", must be Number, Date, DateTime or DateTime64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionSegmentLengthSum, Data>(*arguments[0], arguments));
if (res)
return res;
throw Exception(
"Illegal type " + arguments.front().get()->getName() + " of first argument of aggregate function " + name
+ ", must be Native Unsigned Number",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionSegmentLengthSum(AggregateFunctionFactory & factory)
{
factory.registerFunction("segmentLengthSum", createAggregateFunctionSegmentLengthSum<AggregateFunctionSegmentLengthSumData>);
}
}

View File

@ -0,0 +1,199 @@
#pragma once
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/AggregateFunctionNull.h>
namespace DB
{
template <typename T>
struct AggregateFunctionSegmentLengthSumData
{
using Segment = std::pair<T, T>;
using Segments = PODArrayWithStackMemory<Segment, 64>;
bool sorted = false;
Segments segments;
size_t size() const { return segments.size(); }
void add(T start, T end)
{
if (sorted && segments.size() > 0)
{
sorted = segments.back().first <= start;
}
segments.emplace_back(start, end);
}
void merge(const AggregateFunctionSegmentLengthSumData & other)
{
if (other.segments.empty())
return;
const auto size = segments.size();
segments.insert(std::begin(other.segments), std::end(other.segments));
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
std::stable_sort(std::begin(segments), std::end(segments));
else
{
const auto begin = std::begin(segments);
const auto middle = std::next(begin, size);
const auto end = std::end(segments);
if (!sorted)
std::stable_sort(begin, middle);
if (!other.sorted)
std::stable_sort(middle, end);
std::inplace_merge(begin, middle, end);
}
sorted = true;
}
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(segments), std::end(segments));
sorted = true;
}
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(segments.size(), buf);
for (const auto & time_gap : segments)
{
writeBinary(time_gap.first, buf);
writeBinary(time_gap.second, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
segments.clear();
segments.reserve(size);
T start, end;
for (size_t i = 0; i < size; ++i)
{
readBinary(start, buf);
readBinary(end, buf);
segments.emplace_back(start, end);
}
}
};
template <typename T, typename Data>
class AggregateFunctionSegmentLengthSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionSegmentLengthSum<T, Data>>
{
private:
template <typename TResult>
TResult getSegmentLengthSum(Data & data) const
{
if (data.size() == 0)
return 0;
data.sort();
TResult res = 0;
typename Data::Segment cur_segment = data.segments[0];
for (size_t i = 1; i < data.segments.size(); ++i)
{
if (cur_segment.second < data.segments[i].first)
{
res += cur_segment.second - cur_segment.first;
cur_segment = data.segments[i];
}
else
cur_segment.second = std::max(cur_segment.second, data.segments[i].second);
}
res += cur_segment.second - cur_segment.first;
return res;
}
public:
String getName() const override { return "segmentLengthSum"; }
explicit AggregateFunctionSegmentLengthSum(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionSegmentLengthSum<T, Data>>(arguments, {})
{
}
DataTypePtr getReturnType() const override
{
if constexpr (std::is_floating_point_v<T>)
return std::make_shared<DataTypeFloat64>();
return std::make_shared<DataTypeUInt64>();
}
bool allocatesMemoryInArena() const override { return false; }
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & nested_function,
const DataTypes & arguments,
const Array & params,
const AggregateFunctionProperties & /*properties*/) const override
{
return std::make_shared<AggregateFunctionNullVariadic<false, false, false>>(nested_function, arguments, params);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
auto start = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
auto end = assert_cast<const ColumnVector<T> *>(columns[1])->getData()[row_num];
this->data(place).add(start, end);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if constexpr (std::is_floating_point_v<T>)
assert_cast<ColumnFloat64 &>(to).getData().push_back(getSegmentLengthSum<Float64>(this->data(place)));
else
assert_cast<ColumnUInt64 &>(to).getData().push_back(getSegmentLengthSum<UInt64>(this->data(place)));
}
};
}

View File

@ -114,6 +114,24 @@ static IAggregateFunction * createWithUnsignedIntegerType(const IDataType & argu
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data, typename... TArgs>
static IAggregateFunction * createWithBasicNumberOrDateOrDateTime(const IDataType & argument_type, TArgs &&... args)
{
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return new AggregateFunctionTemplate<TYPE, Data<TYPE>>(std::forward<TArgs>(args)...);
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Date)
return new AggregateFunctionTemplate<UInt16, Data<UInt16>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::DateTime)
return new AggregateFunctionTemplate<UInt32, Data<UInt32>>(std::forward<TArgs>(args)...);
return nullptr;
}
template <template <typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithNumericBasedType(const IDataType & argument_type, TArgs && ... args)
{

View File

@ -0,0 +1,207 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <Common/HashTable/HashMap.h>
#include <common/types.h>
#include <ext/bit_cast.h>
namespace DB
{
/** `bfloat16` is a 16-bit floating point data type that is the same as the corresponding most significant 16 bits of the `float`.
* https://en.wikipedia.org/wiki/Bfloat16_floating-point_format
*
* To calculate quantile, simply convert input value to 16 bit (convert to float, then take the most significant 16 bits),
* and calculate the histogram of these values.
*
* Hash table is the preferred way to store histogram, because the number of distinct values is small:
* ```
* SELECT uniq(bfloat)
* FROM
* (
* SELECT
* number,
* toFloat32(number) AS f,
* bitShiftRight(bitAnd(reinterpretAsUInt32(reinterpretAsFixedString(f)), 4294901760) AS cut, 16),
* reinterpretAsFloat32(reinterpretAsFixedString(cut)) AS bfloat
* FROM numbers(100000000)
* )
*
* uniq(bfloat)
* 2623
*
* ```
* (when increasing the range of values 1000 times, the number of distinct bfloat16 values increases just by 1280).
*
* Then calculate quantile from the histogram.
*
* This sketch is very simple and rough. Its relative precision is constant 1 / 256 = 0.390625%.
*/
template <typename Value>
struct QuantileBFloat16Histogram
{
using BFloat16 = UInt16;
using Weight = UInt64;
/// Make automatic memory for 16 elements to avoid allocations for small states.
/// The usage of trivial hash is ok, because we effectively take logarithm of the values and pathological cases are unlikely.
using Data = HashMapWithStackMemory<BFloat16, Weight, TrivialHash, 4>;
Data data;
void add(const Value & x)
{
add(x, 1);
}
void add(const Value & x, Weight w)
{
if (!isNaN(x))
data[toBFloat16(x)] += w;
}
void merge(const QuantileBFloat16Histogram & rhs)
{
for (const auto & pair : rhs.data)
data[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
}
Value get(Float64 level) const
{
return getImpl<Value>(level);
}
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
{
getManyImpl(levels, indices, size, result);
}
Float64 getFloat(Float64 level) const
{
return getImpl<Float64>(level);
}
void getManyFloat(const Float64 * levels, const size_t * indices, size_t size, Float64 * result) const
{
getManyImpl(levels, indices, size, result);
}
private:
/// Take the most significant 16 bits of the floating point number.
BFloat16 toBFloat16(const Value & x) const
{
return ext::bit_cast<UInt32>(static_cast<Float32>(x)) >> 16;
}
/// Put the bits into most significant 16 bits of the floating point number and fill other bits with zeros.
Float32 toFloat32(const BFloat16 & x) const
{
return ext::bit_cast<Float32>(x << 16);
}
using Pair = PairNoInit<Float32, Weight>;
template <typename T>
T getImpl(Float64 level) const
{
size_t size = data.size();
if (0 == size)
return std::numeric_limits<T>::quiet_NaN();
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
Float64 sum_weight = 0;
Pair * arr_it = array;
for (const auto & pair : data)
{
sum_weight += pair.getMapped();
*arr_it = {toFloat32(pair.getKey()), pair.getMapped()};
++arr_it;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 threshold = std::ceil(sum_weight * level);
Float64 accumulated = 0;
for (const Pair * p = array; p != (array + size); ++p)
{
accumulated += p->second;
if (accumulated >= threshold)
return p->first;
}
return array[size - 1].first;
}
template <typename T>
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, T * result) const
{
size_t size = data.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = std::numeric_limits<T>::quiet_NaN();
return;
}
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
Float64 sum_weight = 0;
Pair * arr_it = array;
for (const auto & pair : data)
{
sum_weight += pair.getMapped();
*arr_it = {toFloat32(pair.getKey()), pair.getMapped()};
++arr_it;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
size_t level_index = 0;
Float64 accumulated = 0;
Float64 threshold = std::ceil(sum_weight * levels[indices[level_index]]);
for (const Pair * p = array; p != (array + size); ++p)
{
accumulated += p->second;
while (accumulated >= threshold)
{
result[indices[level_index]] = p->first;
++level_index;
if (level_index == num_levels)
return;
threshold = std::ceil(sum_weight * levels[indices[level_index]]);
}
}
while (level_index < num_levels)
{
result[indices[level_index]] = array[size - 1].first;
++level_index;
}
}
};
}

View File

@ -62,6 +62,7 @@ void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFact
void registerWindowFunctions(AggregateFunctionFactory & factory);
void registerAggregateFunctionSegmentLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctions()
{
@ -111,6 +112,8 @@ void registerAggregateFunctions()
registerAggregateFunctionStudentTTest(factory);
registerWindowFunctions(factory);
registerAggregateFunctionSegmentLengthSum(factory);
}
{

View File

@ -43,6 +43,7 @@ SRCS(
AggregateFunctionRankCorrelation.cpp
AggregateFunctionResample.cpp
AggregateFunctionRetention.cpp
AggregateFunctionSegmentLengthSum.cpp
AggregateFunctionSequenceMatch.cpp
AggregateFunctionSimpleLinearRegression.cpp
AggregateFunctionSimpleState.cpp

View File

@ -187,6 +187,7 @@ add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProx
add_object_library(clickhouse_interpreters_jit Interpreters/JIT)
add_object_library(clickhouse_columns Columns)
add_object_library(clickhouse_storages Storages)
add_object_library(clickhouse_storages_mysql Storages/MySQL)
add_object_library(clickhouse_storages_distributed Storages/Distributed)
add_object_library(clickhouse_storages_mergetree Storages/MergeTree)
add_object_library(clickhouse_storages_liveview Storages/LiveView)

View File

@ -1,309 +0,0 @@
#pragma once
#include <cstddef>
#include <cstdlib>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_ALLOCATE_MEMORY;
}
/** An array of (almost) unchangeable size:
* the size is specified in the constructor;
* `resize` method removes old data, and necessary only for
* so that you can first create an empty object using the default constructor,
* and then decide on the size.
*
* There is a possibility to not initialize elements by default, but create them inplace.
* Member destructors are called automatically.
*
* `sizeof` is equal to the size of one pointer.
*
* Not exception-safe.
*
* Copying is supported via assign() method. Moving empties the original object.
* That is, it is inconvenient to use this array in many cases.
*
* Designed for situations in which many arrays of the same small size are created,
* but the size is not known at compile time.
* Also gives a significant advantage in cases where it is important that `sizeof` is minimal.
* For example, if arrays are put in an open-addressing hash table with inplace storage of values (like HashMap)
*
* In this case, compared to std::vector:
* - for arrays of 1 element size - an advantage of about 2 times;
* - for arrays of 5 elements - an advantage of about 1.5 times
* (DB::Field, containing UInt64 and String, used as T);
*/
const size_t empty_auto_array_helper = 0;
template <typename T>
class AutoArray
{
public:
/// For deferred creation.
AutoArray()
{
setEmpty();
}
explicit AutoArray(size_t size_)
{
init(size_, false);
}
/** Initializes all elements with a copy constructor with the `value` parameter.
*/
AutoArray(size_t size_, const T & value)
{
init(size_, true);
for (size_t i = 0; i < size_; ++i)
{
new (place(i)) T(value);
}
}
/** `resize` removes all existing items.
*/
void resize(size_t size_, bool dont_init_elems = false)
{
uninit();
init(size_, dont_init_elems);
}
/** Move operations.
*/
AutoArray(AutoArray && src)
{
if (this == &src)
return;
setEmpty();
data_ptr = src.data_ptr;
src.setEmpty();
}
AutoArray & operator= (AutoArray && src)
{
if (this == &src)
return *this;
uninit();
data_ptr = src.data_ptr;
src.setEmpty();
return *this;
}
~AutoArray()
{
uninit();
}
size_t size() const
{
return m_size();
}
bool empty() const
{
return size() == 0;
}
void clear()
{
uninit();
setEmpty();
}
template <typename It>
void assign(It from_begin, It from_end)
{
uninit();
size_t size = from_end - from_begin;
init(size, /* dont_init_elems = */ true);
It it = from_begin;
for (size_t i = 0; i < size; ++i, ++it)
new (place(i)) T(*it);
}
void assign(const AutoArray & from)
{
assign(from.begin(), from.end());
}
/** You can read and modify elements using the [] operator
* only if items were initialized
* (that is, into the constructor was not passed DontInitElemsTag,
* or you initialized them using `place` and `placement new`).
*/
T & operator[](size_t i)
{
return elem(i);
}
const T & operator[](size_t i) const
{
return elem(i);
}
T * data()
{
return elemPtr(0);
}
const T * data() const
{
return elemPtr(0);
}
/** Get the piece of memory in which the element should be located.
* The function is intended to initialize an element,
* which has not yet been initialized
* new (arr.place(i)) T(args);
*/
char * place(size_t i)
{
return data_ptr + sizeof(T) * i;
}
using iterator = T *;
using const_iterator = const T *;
iterator begin() { return elemPtr(0); }
iterator end() { return elemPtr(size()); }
const_iterator begin() const { return elemPtr(0); }
const_iterator end() const { return elemPtr(size()); }
bool operator== (const AutoArray<T> & rhs) const
{
size_t s = size();
if (s != rhs.size())
return false;
for (size_t i = 0; i < s; ++i)
if (elem(i) != rhs.elem(i))
return false;
return true;
}
bool operator!= (const AutoArray<T> & rhs) const
{
return !(*this == rhs);
}
bool operator< (const AutoArray<T> & rhs) const
{
size_t s = size();
size_t rhs_s = rhs.size();
if (s < rhs_s)
return true;
if (s > rhs_s)
return false;
for (size_t i = 0; i < s; ++i)
{
if (elem(i) < rhs.elem(i))
return true;
if (elem(i) > rhs.elem(i))
return false;
}
return false;
}
private:
static constexpr size_t alignment = alignof(T);
/// Bytes allocated to store size of array before data. It is padded to have minimum size as alignment.
/// Padding is at left and the size is stored at right (just before the first data element).
static constexpr size_t prefix_size = std::max(sizeof(size_t), alignment);
char * data_ptr;
size_t & m_size()
{
return reinterpret_cast<size_t *>(data_ptr)[-1];
}
size_t m_size() const
{
return reinterpret_cast<const size_t *>(data_ptr)[-1];
}
T * elemPtr(size_t i)
{
return reinterpret_cast<T *>(data_ptr) + i;
}
const T * elemPtr(size_t i) const
{
return reinterpret_cast<const T *>(data_ptr) + i;
}
T & elem(size_t i)
{
return *elemPtr(i);
}
const T & elem(size_t i) const
{
return *elemPtr(i);
}
void setEmpty()
{
data_ptr = const_cast<char *>(reinterpret_cast<const char *>(&empty_auto_array_helper)) + sizeof(size_t);
}
void init(size_t new_size, bool dont_init_elems)
{
if (!new_size)
{
setEmpty();
return;
}
void * new_data = nullptr;
int res = posix_memalign(&new_data, alignment, prefix_size + new_size * sizeof(T));
if (0 != res)
throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(new_size)),
ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
data_ptr = static_cast<char *>(new_data);
data_ptr += prefix_size;
m_size() = new_size;
if (!dont_init_elems)
for (size_t i = 0; i < new_size; ++i)
new (place(i)) T();
}
void uninit()
{
size_t s = size();
if (s)
{
for (size_t i = 0; i < s; ++i)
elem(i).~T();
data_ptr -= prefix_size;
free(data_ptr);
}
}
};
}

View File

@ -123,7 +123,7 @@ inline bool isWhitespaceASCII(char c)
/// Since |isWhiteSpaceASCII()| is used inside algorithms it's easier to implement another function than add extra argument.
inline bool isWhitespaceASCIIOneLine(char c)
{
return c == ' ' || c == '\t' || c == '\r' || c == '\f' || c == '\v';
return c == ' ' || c == '\t' || c == '\f' || c == '\v';
}
inline bool isControlASCII(char c)

View File

@ -7,9 +7,6 @@ endif()
add_executable (sip_hash_perf sip_hash_perf.cpp)
target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io)
add_executable (auto_array auto_array.cpp)
target_link_libraries (auto_array PRIVATE clickhouse_common_io)
add_executable (small_table small_table.cpp)
target_link_libraries (small_table PRIVATE clickhouse_common_io)

View File

@ -1,197 +0,0 @@
#include <iostream>
#include <iomanip>
#include <map>
#include <pcg_random.hpp>
#include <Core/Field.h>
#include <Common/HashTable/HashMap.h>
#include <Common/AutoArray.h>
#include <IO/WriteHelpers.h>
#include <Common/Stopwatch.h>
int main(int argc, char ** argv)
{
pcg64 rng;
{
size_t n = 10;
using T = std::string;
DB::AutoArray<T> arr(n);
for (size_t i = 0; i < arr.size(); ++i)
arr[i] = "Hello, world! " + DB::toString(i);
for (auto & elem : arr)
std::cerr << elem << std::endl;
}
std::cerr << std::endl;
{
size_t n = 10;
using T = std::string;
using Arr = DB::AutoArray<T>;
Arr arr;
arr.resize(n);
for (size_t i = 0; i < arr.size(); ++i)
arr[i] = "Hello, world! " + DB::toString(i);
for (auto & elem : arr)
std::cerr << elem << std::endl;
std::cerr << std::endl;
Arr arr2 = std::move(arr);
std::cerr << arr.size() << ", " << arr2.size() << std::endl; // NOLINT
for (auto & elem : arr2)
std::cerr << elem << std::endl;
}
std::cerr << std::endl;
{
size_t n = 10;
size_t keys = 10;
using T = std::string;
using Arr = DB::AutoArray<T>;
using Map = std::map<Arr, T>;
Map map;
for (size_t i = 0; i < keys; ++i)
{
Arr key(n);
for (size_t j = 0; j < n; ++j)
key[j] = DB::toString(rng());
map[std::move(key)] = "Hello, world! " + DB::toString(i);
}
for (const auto & kv : map)
{
std::cerr << "[";
for (size_t j = 0; j < n; ++j)
std::cerr << (j == 0 ? "" : ", ") << kv.first[j];
std::cerr << "]";
std::cerr << ":\t" << kv.second << std::endl;
}
std::cerr << std::endl;
Map map2 = std::move(map);
for (const auto & kv : map2)
{
std::cerr << "[";
for (size_t j = 0; j < n; ++j)
std::cerr << (j == 0 ? "" : ", ") << kv.first[j];
std::cerr << "]";
std::cerr << ":\t" << kv.second << std::endl;
}
}
std::cerr << std::endl;
{
size_t n = 10;
size_t keys = 10;
using T = std::string;
using Arr = DB::AutoArray<T>;
using Vec = std::vector<Arr>;
Vec vec;
for (size_t i = 0; i < keys; ++i)
{
Arr key(n);
for (size_t j = 0; j < n; ++j)
key[j] = DB::toString(rng());
vec.push_back(std::move(key));
}
for (const auto & elem : vec)
{
std::cerr << "[";
for (size_t j = 0; j < n; ++j)
std::cerr << (j == 0 ? "" : ", ") << elem[j];
std::cerr << "]" << std::endl;
}
std::cerr << std::endl;
Vec vec2 = std::move(vec);
for (const auto & elem : vec2)
{
std::cerr << "[";
for (size_t j = 0; j < n; ++j)
std::cerr << (j == 0 ? "" : ", ") << elem[j];
std::cerr << "]" << std::endl;
}
}
if (argc == 2 && !strcmp(argv[1], "1"))
{
size_t n = 5;
size_t map_size = 1000000;
using T = DB::Field;
T field = std::string("Hello, world");
using Arr = std::vector<T>;
using Map = HashMap<UInt64, Arr>;
Stopwatch watch;
Map map;
for (size_t i = 0; i < map_size; ++i)
{
Map::LookupResult it;
bool inserted;
map.emplace(rng(), it, inserted);
if (inserted)
{
new (&it->getMapped()) Arr(n);
for (size_t j = 0; j < n; ++j)
(it->getMapped())[j] = field;
}
}
std::cerr << std::fixed << std::setprecision(2)
<< "Vector: Elapsed: " << watch.elapsedSeconds()
<< " (" << map_size / watch.elapsedSeconds() << " rows/sec., "
<< "sizeof(Map::value_type) = " << sizeof(Map::value_type)
<< std::endl;
}
{
size_t n = 10000;
using Arr = DB::AutoArray<std::string>;
Arr arr1(n);
Arr arr2(n);
for (size_t i = 0; i < n; ++i)
{
arr1[i] = "Hello, world! " + DB::toString(i);
arr2[i] = "Goodbye, world! " + DB::toString(i);
}
arr2 = std::move(arr1);
arr1.resize(n); // NOLINT
std::cerr
<< "arr1.size(): " << arr1.size() << ", arr2.size(): " << arr2.size() << std::endl
<< "arr1.data(): " << arr1.data() << ", arr2.data(): " << arr2.data() << std::endl
<< "arr1[0]: " << arr1[0] << ", arr2[0]: " << arr2[0] << std::endl;
}
return 0;
}

View File

@ -61,6 +61,10 @@ static void NO_INLINE testForType(size_t method, size_t rows_size)
test<Key, ::absl::flat_hash_map<Key, UInt64>>(data.data(), data.size(), "Abseil HashMap");
}
else if (method == 3)
{
test<Key, ::absl::flat_hash_map<Key, UInt64, DefaultHash<Key>>>(data.data(), data.size(), "Abseil HashMap with CH Hash");
}
else if (method == 4)
{
test<Key, std::unordered_map<Key, UInt64>>(data.data(), data.size(), "std::unordered_map");
}
@ -81,50 +85,110 @@ static void NO_INLINE testForType(size_t method, size_t rows_size)
* ./integer_hash_tables_benchmark 1 $2 100000000 < $1
* ./integer_hash_tables_benchmark 2 $2 100000000 < $1
* ./integer_hash_tables_benchmark 3 $2 100000000 < $1
* ./integer_hash_tables_benchmark 4 $2 100000000 < $1
*
* Results of this benchmark on hits_100m_obfuscated
* Results of this benchmark on hits_100m_obfuscated X86-64
*
* File hits_100m_obfuscated/201307_1_96_4/WatchID.bin
* CH HashMap: Elapsed: 7.366 (13575745.933 elem/sec.), map size: 99997493
* Google DenseMap: Elapsed: 10.089 (9911817.125 elem/sec.), map size: 99997493
* Abseil HashMap: Elapsed: 9.011 (11097794.073 elem/sec.), map size: 99997493
* std::unordered_map: Elapsed: 44.758 (2234223.189 elem/sec.), map size: 99997493
* CH HashMap: Elapsed: 7.416 (13484217.815 elem/sec.), map size: 99997493
* Google DenseMap: Elapsed: 10.303 (9706022.031 elem/sec.), map size: 99997493
* Abseil HashMap: Elapsed: 9.106 (10982139.229 elem/sec.), map size: 99997493
* Abseil HashMap with CH Hash: Elapsed: 9.221 (10845360.669 elem/sec.), map size: 99997493
* std::unordered_map: Elapsed: 45.213 (2211758.706 elem/sec.), map size: 9999749
*
* File hits_100m_obfuscated/201307_1_96_4/URLHash.bin
* CH HashMap: Elapsed: 2.672 (37421588.347 elem/sec.), map size: 20714865
* Google DenseMap: Elapsed: 3.409 (29333308.209 elem/sec.), map size: 20714865
* Abseil HashMap: Elapsed: 2.778 (36000540.035 elem/sec.), map size: 20714865
* std::unordered_map: Elapsed: 8.643 (11570012.207 elem/sec.), map size: 20714865
* CH HashMap: Elapsed: 2.620 (38168135.308 elem/sec.), map size: 20714865
* Google DenseMap: Elapsed: 3.426 (29189309.058 elem/sec.), map size: 20714865
* Abseil HashMap: Elapsed: 2.788 (35870495.097 elem/sec.), map size: 20714865
* Abseil HashMap with CH Hash: Elapsed: 2.991 (33428850.155 elem/sec.), map size: 20714865
* std::unordered_map: Elapsed: 8.503 (11760331.346 elem/sec.), map size: 20714865
*
* File hits_100m_obfuscated/201307_1_96_4/UserID.bin
* CH HashMap: Elapsed: 2.116 (47267659.076 elem/sec.), map size: 17630976
* Google DenseMap: Elapsed: 2.722 (36740693.786 elem/sec.), map size: 17630976
* Abseil HashMap: Elapsed: 2.597 (38509988.663 elem/sec.), map size: 17630976
* std::unordered_map: Elapsed: 7.327 (13647271.471 elem/sec.), map size: 17630976
* CH HashMap: Elapsed: 2.157 (46352039.753 elem/sec.), map size: 17630976
* Google DenseMap: Elapsed: 2.725 (36694226.782 elem/sec.), map size: 17630976
* Abseil HashMap: Elapsed: 2.590 (38604284.187 elem/sec.), map size: 17630976
* Abseil HashMap with CH Hash: Elapsed: 2.785 (35904856.137 elem/sec.), map size: 17630976
* std::unordered_map: Elapsed: 7.268 (13759557.609 elem/sec.), map size: 17630976
*
* File hits_100m_obfuscated/201307_1_96_4/RegionID.bin
* CH HashMap: Elapsed: 0.201 (498144193.695 elem/sec.), map size: 9040
* Google DenseMap: Elapsed: 0.261 (382656387.016 elem/sec.), map size: 9046
* Abseil HashMap: Elapsed: 0.307 (325874545.117 elem/sec.), map size: 9040
* std::unordered_map: Elapsed: 0.466 (214379083.420 elem/sec.), map size: 9040
* CH HashMap: Elapsed: 0.192 (521583315.810 elem/sec.), map size: 9040
* Google DenseMap: Elapsed: 0.297 (337081407.799 elem/sec.), map size: 9046
* Abseil HashMap: Elapsed: 0.295 (338805623.511 elem/sec.), map size: 9040
* Abseil HashMap with CH Hash: Elapsed: 0.331 (302155391.036 elem/sec.), map size: 9040
* std::unordered_map: Elapsed: 0.455 (219971555.390 elem/sec.), map size: 9040
*
* File hits_100m_obfuscated/201307_1_96_4/CounterID.bin
* CH HashMap: Elapsed: 0.220 (455344735.648 elem/sec.), map size: 6506
* Google DenseMap: Elapsed: 0.297 (336187522.818 elem/sec.), map size: 6506
* Abseil HashMap: Elapsed: 0.307 (325264214.480 elem/sec.), map size: 6506
* std::unordered_map: Elapsed: 0.389 (257195996.114 elem/sec.), map size: 6506
* CH HashMap: Elapsed: 0.217 (460216823.609 elem/sec.), map size: 6506
* Google DenseMap: Elapsed: 0.373 (267838665.098 elem/sec.), map size: 6506
* Abseil HashMap: Elapsed: 0.325 (308124728.989 elem/sec.), map size: 6506
* Abseil HashMap with CH Hash: Elapsed: 0.354 (282167144.801 elem/sec.), map size: 6506
* std::unordered_map: Elapsed: 0.390 (256573354.171 elem/sec.), map size: 6506
*
* File hits_100m_obfuscated/201307_1_96_4/TraficSourceID.bin
* CH HashMap: Elapsed: 0.274 (365196673.729 elem/sec.), map size: 10
* Google DenseMap: Elapsed: 0.782 (127845746.927 elem/sec.), map size: 1565609 /// Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.303 (330461565.053 elem/sec.), map size: 10
* std::unordered_map: Elapsed: 0.843 (118596530.649 elem/sec.), map size: 10
* CH HashMap: Elapsed: 0.246 (406714566.282 elem/sec.), map size: 10
* Google DenseMap: Elapsed: 0.760 (131615151.233 elem/sec.), map size: 1565609 /// Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.309 (324068156.680 elem/sec.), map size: 10
* Abseil HashMap with CH Hash: Elapsed: 0.339 (295108223.814 elem/sec.), map size: 10
* std::unordered_map: Elapsed: 0.811 (123304031.195 elem/sec.), map size: 10
*
* File hits_100m_obfuscated/201307_1_96_4/AdvEngineID.bin
* CH HashMap: Elapsed: 0.160 (623399865.019 elem/sec.), map size: 19
* Google DenseMap: Elapsed: 1.673 (59757144.027 elem/sec.), map size: 32260732 /// Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.297 (336589258.845 elem/sec.), map size: 19
* std::unordered_map: Elapsed: 0.332 (301114451.384 elem/sec.), map size: 19
* CH HashMap: Elapsed: 0.155 (643245257.748 elem/sec.), map size: 19
* Google DenseMap: Elapsed: 1.629 (61395025.417 elem/sec.), map size: 32260732 // Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.292 (342765027.204 elem/sec.), map size: 19
* Abseil HashMap with CH Hash: Elapsed: 0.330 (302822020.210 elem/sec.), map size: 19
* std::unordered_map: Elapsed: 0.308 (325059333.730 elem/sec.), map size: 19
*
*
* Results of this benchmark on hits_100m_obfuscated AARCH64
*
* File hits_100m_obfuscated/201307_1_96_4/WatchID.bin
* CH HashMap: Elapsed: 9.530 (10493528.533 elem/sec.), map size: 99997493
* Google DenseMap: Elapsed: 14.436 (6927091.135 elem/sec.), map size: 99997493
* Abseil HashMap: Elapsed: 16.671 (5998504.085 elem/sec.), map size: 99997493
* Abseil HashMap with CH Hash: Elapsed: 16.803 (5951365.711 elem/sec.), map size: 99997493
* std::unordered_map: Elapsed: 50.805 (1968305.658 elem/sec.), map size: 99997493
*
* File hits_100m_obfuscated/201307_1_96_4/URLHash.bin
* CH HashMap: Elapsed: 3.693 (27076878.092 elem/sec.), map size: 20714865
* Google DenseMap: Elapsed: 5.051 (19796401.694 elem/sec.), map size: 20714865
* Abseil HashMap: Elapsed: 5.617 (17804528.625 elem/sec.), map size: 20714865
* Abseil HashMap with CH Hash: Elapsed: 5.702 (17537013.639 elem/sec.), map size: 20714865
* std::unordered_map: Elapsed: 10.757 (9296040.953 elem/sec.), map size: 2071486
*
* File hits_100m_obfuscated/201307_1_96_4/UserID.bin
* CH HashMap: Elapsed: 2.982 (33535795.695 elem/sec.), map size: 17630976
* Google DenseMap: Elapsed: 3.940 (25381557.959 elem/sec.), map size: 17630976
* Abseil HashMap: Elapsed: 4.493 (22259078.458 elem/sec.), map size: 17630976
* Abseil HashMap with CH Hash: Elapsed: 4.596 (21759738.710 elem/sec.), map size: 17630976
* std::unordered_map: Elapsed: 9.035 (11067903.596 elem/sec.), map size: 17630976
*
* File hits_100m_obfuscated/201307_1_96_4/RegionID.bin
* CH HashMap: Elapsed: 0.302 (331026285.361 elem/sec.), map size: 9040
* Google DenseMap: Elapsed: 0.623 (160419421.840 elem/sec.), map size: 9046
* Abseil HashMap: Elapsed: 0.981 (101971186.758 elem/sec.), map size: 9040
* Abseil HashMap with CH Hash: Elapsed: 0.991 (100932993.199 elem/sec.), map size: 9040
* std::unordered_map: Elapsed: 0.809 (123541402.715 elem/sec.), map size: 9040
*
* File hits_100m_obfuscated/201307_1_96_4/CounterID.bin
* CH HashMap: Elapsed: 0.343 (291821742.078 elem/sec.), map size: 6506
* Google DenseMap: Elapsed: 0.718 (139191105.450 elem/sec.), map size: 6506
* Abseil HashMap: Elapsed: 1.019 (98148285.278 elem/sec.), map size: 6506
* Abseil HashMap with CH Hash: Elapsed: 1.048 (95446843.667 elem/sec.), map size: 6506
* std::unordered_map: Elapsed: 0.701 (142701070.085 elem/sec.), map size: 6506
*
* File hits_100m_obfuscated/201307_1_96_4/TraficSourceID.bin
* CH HashMap: Elapsed: 0.376 (265905243.103 elem/sec.), map size: 10
* Google DenseMap: Elapsed: 1.309 (76420707.298 elem/sec.), map size: 1565609 /// Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.955 (104668109.775 elem/sec.), map size: 10
* Abseil HashMap with CH Hash: Elapsed: 0.967 (103456305.391 elem/sec.), map size: 10
* std::unordered_map: Elapsed: 1.241 (80591305.890 elem/sec.), map size: 10
*
* File hits_100m_obfuscated/201307_1_96_4/AdvEngineID.bin
* CH HashMap: Elapsed: 0.213 (470208130.105 elem/sec.), map size: 19
* Google DenseMap: Elapsed: 2.525 (39607131.523 elem/sec.), map size: 32260732 /// Broken because there is 0 key in dataset
* Abseil HashMap: Elapsed: 0.950 (105233678.618 elem/sec.), map size: 19
* Abseil HashMap with CH Hash: Elapsed: 0.962 (104001230.717 elem/sec.), map size: 19
* std::unordered_map: Elapsed: 0.585 (171059989.837 elem/sec.), map size: 19
*/
int main(int argc, char ** argv)

View File

@ -81,10 +81,41 @@ struct NetworkInterfaces
bool isLocalAddress(const Poco::Net::IPAddress & address)
{
/** 127.0.0.1 is treat as local address unconditionally.
* ::1 is also treat as local address unconditionally.
*
* 127.0.0.{2..255} are not treat as local addresses, because they are used in tests
* to emulate distributed queries across localhost.
*
* But 127.{0,1}.{0,1}.{0,1} are treat as local addresses,
* because they are used in Debian for localhost.
*/
if (address.isLoopback())
{
if (address.family() == Poco::Net::AddressFamily::IPv4)
{
/// The address is located in memory in big endian form (network byte order).
const unsigned char * digits = static_cast<const unsigned char *>(address.addr());
if (digits[0] == 127
&& digits[1] <= 1
&& digits[2] <= 1
&& digits[3] <= 1)
{
return true;
}
}
else if (address.family() == Poco::Net::AddressFamily::IPv6)
{
return true;
}
}
NetworkInterfaces interfaces;
return interfaces.hasAddress(address);
}
bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port)
{
return clickhouse_port == address.port() && isLocalAddress(address.host());

View File

@ -11,9 +11,30 @@ TEST(LocalAddress, SmokeTest)
std::string address_str;
DB::readString(address_str, cmd->out);
cmd->wait();
std::cerr << "Got Address:" << address_str << std::endl;
std::cerr << "Got Address: " << address_str << std::endl;
Poco::Net::IPAddress address(address_str);
EXPECT_TRUE(DB::isLocalAddress(address));
}
TEST(LocalAddress, Localhost)
{
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"127.0.0.1"}));
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"127.0.1.1"}));
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"127.1.1.1"}));
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"127.1.0.1"}));
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"127.1.0.0"}));
EXPECT_TRUE(DB::isLocalAddress(Poco::Net::IPAddress{"::1"}));
/// Make sure we don't mess with the byte order.
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"1.0.0.127"}));
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"1.1.1.127"}));
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"0.0.0.0"}));
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"::"}));
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"::2"}));
/// See the comment in the implementation of isLocalAddress.
EXPECT_FALSE(DB::isLocalAddress(Poco::Net::IPAddress{"127.0.0.2"}));
}

View File

@ -721,6 +721,9 @@ private:
#undef DBMS_MIN_FIELD_SIZE
using Row = std::vector<Field>;
template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; };
template <> struct Field::TypeToEnum<UInt64> { static const Types::Which value = Types::UInt64; };
template <> struct Field::TypeToEnum<UInt128> { static const Types::Which value = Types::UInt128; };

View File

@ -452,7 +452,7 @@ namespace MySQLReplication
UInt32 number_columns;
String schema;
String table;
std::vector<Field> rows;
Row rows;
RowsEvent(std::shared_ptr<TableMapEvent> table_map_, EventHeader && header_, const RowsEventHeader & rows_header)
: EventBase(std::move(header_)), number_columns(0), table_map(table_map_)

View File

@ -1,18 +0,0 @@
#pragma once
#include <vector>
#include <Common/AutoArray.h>
#include <Core/Field.h>
namespace DB
{
/** The data type for representing one row of the table in the RAM.
* Warning! It is preferable to store column blocks instead of single rows. See Block.h
*/
using Row = AutoArray<Field>;
}

View File

@ -1,6 +1,5 @@
#pragma once
#include <Core/Row.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>

View File

@ -193,7 +193,7 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view
size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions;
const auto parse_value = array_info[idx].pqxx_parser;
std::vector<std::vector<Field>> dimensions(expected_dimensions + 1);
std::vector<Row> dimensions(expected_dimensions + 1);
while (parsed.first != pqxx::array_parser::juncture::done)
{

View File

@ -20,6 +20,7 @@
# include <Parsers/parseQuery.h>
# include <Parsers/queryToString.h>
# include <Storages/StorageMySQL.h>
# include <Storages/MySQL/MySQLSettings.h>
# include <Common/escapeForFileName.h>
# include <Common/parseAddress.h>
# include <Common/setThreadName.h>
@ -253,12 +254,13 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(
std::move(mysql_pool),
database_name_in_mysql,
table_name,
false,
"",
/* replace_query_ */ false,
/* on_duplicate_clause = */ "",
ColumnsDescription{columns_name_and_type},
ConstraintsDescription{},
String{},
getContext()));
getContext(),
MySQLSettings{}));
}
}

View File

@ -477,7 +477,7 @@ static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value,
template <bool assert_nullable = false>
static void writeFieldsToColumn(
IColumn & column_to, const std::vector<Field> & rows_data, size_t column_index, const std::vector<bool> & mask, ColumnUInt8 * null_map_column = nullptr)
IColumn & column_to, const Row & rows_data, size_t column_index, const std::vector<bool> & mask, ColumnUInt8 * null_map_column = nullptr)
{
if (ColumnNullable * column_nullable = typeid_cast<ColumnNullable *>(&column_to))
writeFieldsToColumn<true>(column_nullable->getNestedColumn(), rows_data, column_index, mask, &column_nullable->getNullMapColumn());
@ -599,7 +599,7 @@ static void writeFieldsToColumn(
}
template <Int8 sign>
static size_t onWriteOrDeleteData(const std::vector<Field> & rows_data, Block & buffer, size_t version)
static size_t onWriteOrDeleteData(const Row & rows_data, Block & buffer, size_t version)
{
size_t prev_bytes = buffer.bytes();
for (size_t column = 0; column < buffer.columns() - 2; ++column)
@ -623,7 +623,7 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple
return false;
}
static inline size_t onUpdateData(const std::vector<Field> & rows_data, Block & buffer, size_t version, const std::vector<size_t> & sorting_columns_index)
static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector<size_t> & sorting_columns_index)
{
if (rows_data.size() % 2 != 0)
throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR);

View File

@ -1,6 +1,5 @@
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Core/Row.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsMiscellaneous.h>

View File

@ -34,7 +34,7 @@ public:
FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; }
private:
std::vector<Field> row;
Row row;
SortDescription description;
};

View File

@ -1,7 +1,6 @@
#include <optional>
#include <Core/Field.h>
#include <Core/Row.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>

View File

@ -12,7 +12,6 @@
#define DBMS_HASH_MAP_COUNT_COLLISIONS
*/
#include <common/types.h>
#include <Core/Row.h>
#include <IO/ReadBufferFromFile.h>
#include <Compression/CompressedReadBuffer.h>
#include <Common/HashTable/HashMap.h>
@ -27,19 +26,6 @@
* This is important, because if you run all the tests one by one, the results will be incorrect.
* (Due to the peculiarities of the work of the allocator, the first test takes advantage.)
*
* Depending on USE_AUTO_ARRAY, one of the structures is selected as the value.
* USE_AUTO_ARRAY = 0 - uses std::vector (hard-copy structure, sizeof = 24 bytes).
* USE_AUTO_ARRAY = 1 - uses AutoArray (a structure specially designed for such cases, sizeof = 8 bytes).
*
* That is, the test also allows you to compare AutoArray and std::vector.
*
* If USE_AUTO_ARRAY = 0, then HashMap confidently overtakes all.
* If USE_AUTO_ARRAY = 1, then HashMap is slightly less serious (20%) ahead of google::dense_hash_map.
*
* When using HashMap, AutoArray has a rather serious (40%) advantage over std::vector.
* And when using other hash tables, AutoArray even more seriously overtakes std::vector
* (up to three and a half times in the case of std::unordered_map and google::sparse_hash_map).
*
* HashMap, unlike google::dense_hash_map, much more depends on the quality of the hash function.
*
* PS. Measure everything yourself, otherwise I'm almost confused.
@ -49,9 +35,6 @@
* But in this test, there was something similar to the old scenario of using hash tables in the aggregation.
*/
#define USE_AUTO_ARRAY 0
struct AlternativeHash
{
size_t operator() (UInt64 x) const
@ -85,12 +68,7 @@ int main(int argc, char ** argv)
using namespace DB;
using Key = UInt64;
#if USE_AUTO_ARRAY
using Value = AutoArray<IAggregateFunction*>;
#else
using Value = std::vector<IAggregateFunction*>;
#endif
size_t n = argc < 2 ? 10000000 : std::stol(argv[1]);
//size_t m = std::stol(argv[2]);
@ -119,13 +97,8 @@ int main(int argc, char ** argv)
INIT
#ifndef USE_AUTO_ARRAY
#undef INIT
#define INIT
#endif
Row row(1);
row[0] = UInt64(0);
std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl;

View File

@ -95,7 +95,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (tables())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FROM " << (s.hilite ? hilite_none : "");
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FROM" << (s.hilite ? hilite_none : "");
tables()->formatImpl(s, state, frame);
}

View File

@ -35,24 +35,24 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F
if (mode == Mode::Unspecified)
return "";
else if (mode == Mode::ALL)
return "ALL";
return " ALL";
else
return "DISTINCT";
return " DISTINCT";
};
for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it)
{
if (it != list_of_selects->children.begin())
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION "
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION"
<< mode_to_str((is_normalized) ? union_mode : list_of_modes[it - list_of_selects->children.begin() - 1])
<< (settings.hilite ? hilite_none : "");
if (auto * node = (*it)->as<ASTSelectWithUnionQuery>())
{
settings.ostr << settings.nl_or_ws << indent_str;
if (node->list_of_selects->children.size() == 1)
{
if (it != list_of_selects->children.begin())
settings.ostr << settings.nl_or_ws;
(node->list_of_selects->children.at(0))->formatImpl(settings, state, frame);
}
else

View File

@ -29,6 +29,11 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const
void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
/// NOTE: due to trickery of filling cte_name (in interpreters) it is hard
/// to print it w/o newline (for !oneline case), since if nl_or_ws
/// prepended here, then formatting will be incorrect with alias:
///
/// (select 1 in ((select 1) as sub))
if (!cte_name.empty())
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
@ -40,7 +45,7 @@ void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, Format
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
std::string nl_or_nothing = settings.one_line ? "" : "\n";
settings.ostr << nl_or_nothing << indent_str << "(" << nl_or_nothing;
settings.ostr << "(" << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
++frame_nested.indent;

View File

@ -109,14 +109,17 @@ void ASTTableExpression::formatImpl(const FormatSettings & settings, FormatState
if (database_and_table_name)
{
settings.ostr << " ";
database_and_table_name->formatImpl(settings, state, frame);
}
else if (table_function)
{
settings.ostr << " ";
table_function->formatImpl(settings, state, frame);
}
else if (subquery)
{
settings.ostr << settings.nl_or_ws << indent_str;
subquery->formatImpl(settings, state, frame);
}
@ -142,9 +145,15 @@ void ASTTableExpression::formatImpl(const FormatSettings & settings, FormatState
}
void ASTTableJoin::formatImplBeforeTable(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTTableJoin::formatImplBeforeTable(const FormatSettings & settings, FormatState &, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
if (kind != Kind::Comma)
{
settings.ostr << settings.nl_or_ws << indent_str;
}
switch (locality)
{
@ -241,6 +250,7 @@ void ASTArrayJoin::formatImpl(const FormatSettings & settings, FormatState & sta
frame.expression_list_prepend_whitespace = true;
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< settings.nl_or_ws
<< (kind == Kind::Left ? "LEFT " : "") << "ARRAY JOIN" << (settings.hilite ? hilite_none : "");
settings.one_line
@ -254,10 +264,7 @@ void ASTTablesInSelectQueryElement::formatImpl(const FormatSettings & settings,
if (table_expression)
{
if (table_join)
{
table_join->as<ASTTableJoin &>().formatImplBeforeTable(settings, state, frame);
settings.ostr << " ";
}
table_expression->formatImpl(settings, state, frame);
@ -275,13 +282,8 @@ void ASTTablesInSelectQuery::formatImpl(const FormatSettings & settings, FormatS
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << settings.nl_or_ws << indent_str;
(*it)->formatImpl(settings, state, frame);
}
for (const auto & child : children)
child->formatImpl(settings, state, frame);
}
}

View File

@ -16,8 +16,11 @@ ASTPtr ASTWithElement::clone() const
void ASTWithElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << (settings.hilite ? hilite_none : "");
settings.ostr << settings.nl_or_ws << indent_str;
dynamic_cast<const ASTWithAlias &>(*subquery).formatImplWithoutAlias(settings, state, frame);
}

View File

@ -344,7 +344,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
if (target.isEnum())
{
const auto & enum_type = dynamic_cast<const IDataTypeEnum &>(*target_type);
std::vector<Field> symbol_mapping;
Row symbol_mapping;
for (size_t i = 0; i < root_node->names(); i++)
{
symbol_mapping.push_back(enum_type.castToValue(root_node->nameAt(i)));

View File

@ -12,6 +12,7 @@
#include <DataTypes/NestedUtils.h>
#include <IO/WriteHelpers.h>
namespace DB
{
@ -109,6 +110,9 @@ static bool isInPartitionKey(const std::string & column_name, const Names & part
return is_in_partition_key != partition_key_columns.end();
}
using Row = std::vector<Field>;
/// Returns true if merge result is not empty
static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc,
Row & row, const ColumnRawPtrs & raw_columns, size_t row_number)

View File

@ -2,7 +2,7 @@
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
#include <Processors/Merges/Algorithms/MergedData.h>
#include <Core/Row.h>
namespace DB
{

View File

@ -68,9 +68,6 @@ void readHeaders(
if (in.eof())
throw Poco::Net::MessageException("Field is invalid");
if (value.empty())
throw Poco::Net::MessageException("Field value is empty");
if (ch == '\n')
throw Poco::Net::MessageException("No CRLF found");

View File

@ -2,7 +2,6 @@
#include <DataStreams/IBlockInputStream.h>
#include <Core/Row.h>
#include <Core/Block.h>
#include <common/types.h>
#include <Core/NamesAndTypes.h>

View File

@ -211,9 +211,12 @@ namespace
virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0;
virtual void insertUInt64Column(const ColumnPtr & column, const String & name) = 0;
virtual void insertUUIDColumn(const ColumnPtr & column, const String & name) = 0;
virtual void
insertPartitionValueColumn(size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name)
= 0;
virtual void insertPartitionValueColumn(
size_t rows,
const Row & partition_value,
const DataTypePtr & partition_value_type,
const String & name) = 0;
};
}
@ -358,8 +361,8 @@ namespace
columns.push_back(column);
}
void
insertPartitionValueColumn(size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final
void insertPartitionValueColumn(
size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final
{
ColumnPtr column;
if (rows)

View File

@ -79,7 +79,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri
new_data_part->uuid = uuid;
new_data_part->setColumns(columns);
new_data_part->partition.value.assign(partition.value);
new_data_part->partition.value = partition.value;
new_data_part->minmax_idx = minmax_idx;
if (disk->exists(destination_path))

View File

@ -150,7 +150,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned.
{
result.emplace_back(Block(block), Row());
result.emplace_back(Block(block), Row{});
return result;
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <Core/Row.h>
#include <IO/WriteBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h>

View File

@ -1,9 +1,10 @@
#pragma once
#include <Core/Row.h>
#include <common/types.h>
#include <Disks/IDisk.h>
#include <IO/WriteBuffer.h>
#include <Core/Field.h>
namespace DB
{
@ -38,7 +39,7 @@ public:
void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
void assign(const MergeTreePartition & other) { value.assign(other.value); }
void assign(const MergeTreePartition & other) { value = other.value; }
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row);
};

View File

@ -0,0 +1,42 @@
#include <Storages/MySQL/MySQLSettings.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS)
void MySQLSettings::loadFromQuery(ASTStorage & storage_def)
{
if (storage_def.settings)
{
try
{
applyChanges(storage_def.settings->changes);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
e.addMessage("for storage " + storage_def.engine->name);
throw;
}
}
else
{
auto settings_ast = std::make_shared<ASTSetQuery>();
settings_ast->is_standalone = false;
storage_def.set(storage_def.settings, settings_ast);
}
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Core/Defines.h>
#include <Core/BaseSettings.h>
namespace Poco::Util
{
class AbstractConfiguration;
}
namespace DB
{
class ASTStorage;
#define LIST_OF_MYSQL_SETTINGS(M) \
M(UInt64, connection_pool_size, 16, "Size of connection pool (if all connections are in use, the query will wait until some connection will be freed).", 0) \
M(UInt64, connection_max_tries, 3, "Number of retries for pool with failover", 0) \
M(Bool, connection_auto_close, true, "Auto-close connection after query execution, i.e. disable connection reuse.", 0) \
DECLARE_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS)
/** Settings for the MySQL family of engines.
*/
struct MySQLSettings : public BaseSettings<MySQLSettingsTraits>
{
void loadFromQuery(ASTStorage & storage_def);
};
}

View File

@ -12,6 +12,7 @@
#include <Processors/Pipe.h>
#include <Common/parseRemoteDescription.h>
#include <Storages/StorageMySQL.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <Storages/StoragePostgreSQL.h>
#include <Storages/StorageURL.h>
#include <common/logger_useful.h>
@ -79,7 +80,8 @@ StorageExternalDistributed::StorageExternalDistributed(
columns_,
constraints_,
String{},
context);
context,
MySQLSettings{});
break;
}
#endif

View File

@ -15,6 +15,7 @@
#include <IO/Operators.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTCreateQuery.h>
#include <mysqlxx/Transaction.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Pipe.h>
@ -50,13 +51,15 @@ StorageMySQL::StorageMySQL(
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_)
ContextPtr context_,
const MySQLSettings & mysql_settings_)
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
, remote_database_name(remote_database_name_)
, remote_table_name(remote_table_name_)
, replace_query{replace_query_}
, on_duplicate_clause{on_duplicate_clause_}
, mysql_settings(mysql_settings_)
, pool(std::make_shared<mysqlxx::PoolWithFailover>(pool_))
{
StorageInMemoryMetadata storage_metadata;
@ -98,7 +101,8 @@ Pipe StorageMySQL::read(
}
StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), true, false);
StreamSettings mysql_input_stream_settings(context_->getSettingsRef(),
mysql_settings.connection_auto_close);
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MySQLWithFailoverBlockInputStream>(pool, query, sample_block, mysql_input_stream_settings)));
}
@ -250,8 +254,22 @@ void registerStorageMySQL(StorageFactory & factory)
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = args.getContext()->getSettingsRef().glob_expansion_max_elements;
/// TODO: move some arguments from the arguments to the SETTINGS.
MySQLSettings mysql_settings;
if (args.storage_def->settings)
{
mysql_settings.loadFromQuery(*args.storage_def);
}
if (!mysql_settings.connection_pool_size)
throw Exception("connection_pool_size cannot be zero.", ErrorCodes::BAD_ARGUMENTS);
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
mysqlxx::PoolWithFailover pool(remote_database, addresses, username, password);
mysqlxx::PoolWithFailover pool(remote_database, addresses,
username, password,
MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
mysql_settings.connection_pool_size,
mysql_settings.connection_max_tries);
bool replace_query = false;
std::string on_duplicate_clause;
@ -275,9 +293,11 @@ void registerStorageMySQL(StorageFactory & factory)
args.columns,
args.constraints,
args.comment,
args.getContext());
args.getContext(),
mysql_settings);
},
{
.supports_settings = true,
.source_access_type = AccessType::MYSQL,
});
}

View File

@ -9,6 +9,7 @@
#include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <mysqlxx/PoolWithFailover.h>
@ -33,7 +34,8 @@ public:
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_);
ContextPtr context_,
const MySQLSettings & mysql_settings_);
std::string getName() const override { return "MySQL"; }
@ -56,6 +58,8 @@ private:
bool replace_query;
std::string on_duplicate_clause;
MySQLSettings mysql_settings;
mysqlxx::PoolWithFailoverPtr pool;
};

View File

@ -112,6 +112,7 @@ SRCS(
MergeTree/localBackup.cpp
MergeTree/registerStorageMergeTree.cpp
MutationCommands.cpp
MySQL/MySQLSettings.cpp
PartitionCommands.cpp
ProjectionsDescription.cpp
ReadInOrderOptimizer.cpp

View File

@ -15,6 +15,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/StorageMySQL.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionMySQL.h>
@ -107,7 +108,8 @@ StoragePtr TableFunctionMySQL::executeImpl(
columns,
ConstraintsDescription{},
String{},
context);
context,
MySQLSettings{});
pool.reset();

View File

@ -314,11 +314,12 @@ def colored(text, args, color=None, on_color=None, attrs=None):
SERVER_DIED = False
exit_code = 0
stop_time = None
queue = multiprocessing.Queue(maxsize=1)
# def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total):
def run_tests_array(all_tests_with_params):
all_tests, suite, suite_dir, suite_tmp_dir = all_tests_with_params
all_tests, num_tests, suite, suite_dir, suite_tmp_dir = all_tests_with_params
global exit_code
global SERVER_DIED
global stop_time
@ -348,10 +349,21 @@ def run_tests_array(all_tests_with_params):
else:
return ''
if all_tests:
print(f"\nRunning {len(all_tests)} {suite} tests ({multiprocessing.current_process().name}).\n")
if num_tests > 0:
about = 'about ' if is_concurrent else ''
print(f"\nRunning {about}{num_tests} {suite} tests ({multiprocessing.current_process().name}).\n")
while True:
if is_concurrent:
case = queue.get()
if not case:
break
else:
if all_tests:
case = all_tests.pop(0)
else:
break
for case in all_tests:
if SERVER_DIED:
stop_tests()
break
@ -677,6 +689,47 @@ def collect_build_flags(client):
return result
def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, parallel):
if jobs > 1 and len(parallel_tests) > 0:
print("Found", len(parallel_tests), "parallel tests and", len(sequential_tests), "sequential tests")
run_n, run_total = parallel.split('/')
run_n = float(run_n)
run_total = float(run_total)
tests_n = len(parallel_tests)
if run_total > tests_n:
run_total = tests_n
if jobs > tests_n:
jobs = tests_n
if jobs > run_total:
run_total = jobs
batch_size = max(1, len(parallel_tests) // jobs)
parallel_tests_array = []
for _ in range(jobs):
parallel_tests_array.append((None, batch_size, suite, suite_dir, suite_tmp_dir))
with closing(multiprocessing.Pool(processes=jobs)) as pool:
pool.map_async(run_tests_array, parallel_tests_array)
for suit in parallel_tests:
queue.put(suit)
for _ in range(jobs):
queue.put(None)
queue.close()
pool.join()
run_tests_array((sequential_tests, len(sequential_tests), suite, suite_dir, suite_tmp_dir))
return len(sequential_tests) + len(parallel_tests)
else:
num_tests = len(all_tests)
run_tests_array((all_tests, num_tests, suite, suite_dir, suite_tmp_dir))
return num_tests
def main(args):
global SERVER_DIED
global stop_time
@ -840,34 +893,8 @@ def main(args):
else:
parallel_tests.append(test)
if jobs > 1 and len(parallel_tests) > 0:
print("Found", len(parallel_tests), "parallel tests and", len(sequential_tests), "sequential tests")
run_n, run_total = args.parallel.split('/')
run_n = float(run_n)
run_total = float(run_total)
tests_n = len(parallel_tests)
if run_total > tests_n:
run_total = tests_n
if jobs > tests_n:
jobs = tests_n
if jobs > run_total:
run_total = jobs
# Create two batches per process for more uniform execution time.
batch_size = max(1, len(parallel_tests) // (jobs * 2))
parallel_tests_array = []
for i in range(0, len(parallel_tests), batch_size):
parallel_tests_array.append((parallel_tests[i:i+batch_size], suite, suite_dir, suite_tmp_dir))
with closing(multiprocessing.Pool(processes=jobs)) as pool:
pool.map(run_tests_array, parallel_tests_array)
run_tests_array((sequential_tests, suite, suite_dir, suite_tmp_dir))
total_tests_run += len(sequential_tests) + len(parallel_tests)
else:
run_tests_array((all_tests, suite, suite_dir, suite_tmp_dir))
total_tests_run += len(all_tests)
total_tests_run += do_run_tests(
jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, args.parallel)
if args.hung_check:

View File

@ -0,0 +1,29 @@
<test>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<substitutions>
<substitution>
<name>key</name>
<values>
<value>SearchEngineID</value>
<value>RegionID</value>
<value>SearchPhrase</value>
<value>ClientIP</value>
</values>
</substitution>
<substitution>
<name>func</name>
<values>
<value>quantile</value>
<value>quantileExact</value>
<value>quantileTDigest</value>
<value>quantileTiming</value>
<value>quantileBFloat16</value>
</values>
</substitution>
</substitutions>
<query>SELECT {key} AS k, {func}(ResolutionWidth) FROM hits_100m_single GROUP BY k FORMAT Null</query>
</test>

View File

@ -1,12 +1,9 @@
<test>
<preconditions>
<table_exists>hits_100m_single</table_exists>
<ram_size>30000000000</ram_size>
</preconditions>
<settings>
<max_memory_usage>30000000000</max_memory_usage>
<!--
@ -36,7 +33,7 @@
<value>SearchPhrase</value>
<value>ClientIP</value>
</values>
</substitution>
</substitution>
<substitution>
<name>func</name>
<values>

View File

@ -1,6 +1,6 @@
SET max_block_size = 1000;
SELECT number FROM
SELECT number FROM
(
SELECT * FROM system.numbers LIMIT 10000
)

View File

@ -2,7 +2,7 @@ SET any_join_distinct_right_table_keys = 1;
SELECT
floor((ReferrerTimestamp - InstallTimestamp) / 86400) AS DaysSinceInstallations
FROM
FROM
(
SELECT 6534090703218709881 AS DeviceIDHash, 1458586663 AS InstallTimestamp
UNION ALL SELECT 2697418689476658272, 1458561552

View File

@ -11,7 +11,7 @@ INSERT INTO transactions VALUES ('facebook.com'), ('yandex.ru'), ('baidu.com');
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -36,7 +36,7 @@ FORMAT JSONEachRow;
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -63,7 +63,7 @@ SELECT DISTINCT * FROM
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -88,7 +88,7 @@ UNION ALL
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -116,7 +116,7 @@ SELECT
sum(total_count) AS total,
sum(facebookHits) AS facebook,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -142,7 +142,7 @@ SELECT
sum(total_count) AS total,
max(facebookHits) AS facebook,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -170,7 +170,7 @@ SELECT * FROM
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -195,7 +195,7 @@ ALL FULL OUTER JOIN
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -224,7 +224,7 @@ SELECT total FROM
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -249,7 +249,7 @@ ALL FULL OUTER JOIN
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -278,7 +278,7 @@ SELECT domain FROM
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,
@ -303,7 +303,7 @@ ALL FULL OUTER JOIN
SELECT
sum(total_count) AS total,
domain
FROM
FROM
(
SELECT
COUNT(*) AS total_count,

View File

@ -5,7 +5,7 @@
2000-01-01 1 test string 1 1
-------Forbid push down-------
SELECT count()
FROM
FROM
(
SELECT
[number] AS a,
@ -21,11 +21,11 @@ WHERE NOT ignore(a + b)
SELECT
a,
b
FROM
FROM
(
SELECT 1 AS a
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT
1 AS a,
@ -35,13 +35,13 @@ WHERE b = 0
SELECT
a,
b
FROM
FROM
(
SELECT
1 AS a,
1 AS b
)
ANY RIGHT JOIN
ANY RIGHT JOIN
(
SELECT 1 AS a
) USING (a)
@ -49,11 +49,11 @@ WHERE b = 0
SELECT
a,
b
FROM
FROM
(
SELECT 1 AS a
)
ANY FULL OUTER JOIN
ANY FULL OUTER JOIN
(
SELECT
1 AS a,
@ -63,26 +63,26 @@ WHERE b = 0
SELECT
a,
b
FROM
FROM
(
SELECT
1 AS a,
1 AS b
)
ANY FULL OUTER JOIN
ANY FULL OUTER JOIN
(
SELECT 1 AS a
) USING (a)
WHERE b = 0
-------Need push down-------
SELECT toString(value) AS value
FROM
FROM
(
SELECT 1 AS value
)
1
SELECT id
FROM
FROM
(
SELECT 1 AS id
UNION ALL
@ -92,7 +92,7 @@ FROM
WHERE id = 1
1
SELECT id
FROM
FROM
(
SELECT arrayJoin([1, 2, 3]) AS id
WHERE id = 1
@ -100,7 +100,7 @@ FROM
WHERE id = 1
1
SELECT id
FROM
FROM
(
SELECT arrayJoin([1, 2, 3]) AS id
WHERE id = 1
@ -110,7 +110,7 @@ WHERE id = 1
SELECT
id,
subquery
FROM
FROM
(
SELECT
1 AS id,
@ -122,7 +122,7 @@ WHERE subquery = 1
SELECT
a,
b
FROM
FROM
(
SELECT
toUInt64(sum(id) AS b) AS a,
@ -137,7 +137,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -156,7 +156,7 @@ WHERE id = 1
SELECT
a,
b
FROM
FROM
(
SELECT
toUInt64(sum(id) AS b) AS a,
@ -171,7 +171,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -188,14 +188,14 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -214,14 +214,14 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -240,7 +240,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -257,14 +257,14 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -283,7 +283,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -300,14 +300,14 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -325,7 +325,7 @@ SELECT
id,
date,
value
FROM
FROM
(
SELECT
id,
@ -344,7 +344,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -373,7 +373,7 @@ SELECT
date,
name,
value
FROM
FROM
(
SELECT
date,
@ -383,7 +383,7 @@ FROM
FROM test_00597
WHERE id = 1
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT id
FROM test_00597
@ -395,11 +395,11 @@ SELECT
date,
name,
value
FROM
FROM
(
SELECT toInt8(1) AS id
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT
date,
@ -411,7 +411,7 @@ ANY LEFT JOIN
WHERE value = 1
1 2000-01-01 test string 1 1
SELECT value
FROM
FROM
(
SELECT toInt8(1) AS id
)
@ -423,7 +423,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -433,7 +433,7 @@ FROM
date,
name,
value
FROM
FROM
(
SELECT
date,
@ -443,7 +443,7 @@ FROM
FROM test_00597
WHERE id = 1
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT id
FROM test_00597
@ -460,7 +460,7 @@ SELECT
b.date,
b.name,
b.value
FROM
FROM
(
SELECT
date,
@ -469,7 +469,7 @@ FROM
value
FROM test_00597
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT
date,
@ -485,7 +485,7 @@ SELECT
date,
name,
value
FROM
FROM
(
SELECT
toInt8(1) AS id,
@ -493,7 +493,7 @@ FROM
FROM system.numbers
LIMIT 1
)
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT
date,
@ -513,7 +513,7 @@ SELECT
`b.id`,
`b.name`,
`b.value`
FROM
FROM
(
SELECT
date,
@ -524,7 +524,7 @@ FROM
b.id,
b.name,
b.value
FROM
FROM
(
SELECT
date,
@ -534,7 +534,7 @@ FROM
FROM test_00597
WHERE id = 1
) AS a
ANY LEFT JOIN
ANY LEFT JOIN
(
SELECT
date,
@ -555,7 +555,7 @@ SELECT
r.date,
r.name,
r.value
FROM
FROM
(
SELECT
date,
@ -564,14 +564,14 @@ FROM
value
FROM test_00597
)
SEMI LEFT JOIN
SEMI LEFT JOIN
(
SELECT
date,
id,
name,
value
FROM
FROM
(
SELECT
date,
@ -586,7 +586,7 @@ SEMI LEFT JOIN
WHERE r.id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT value + t1.value AS expr
FROM
FROM
(
SELECT
value,

View File

@ -1 +1 @@
CREATE VIEW default.test_view_00599\n(\n `id` UInt64\n) AS\nSELECT *\nFROM default.test_00599\nWHERE id = \n(\n SELECT 1\n)
CREATE VIEW default.test_view_00599\n(\n `id` UInt64\n) AS\nSELECT *\nFROM default.test_00599\nWHERE id = (\n SELECT 1\n)

View File

@ -5,7 +5,7 @@ SELECT
uniqExact(x) AS u,
uniqExactIf(x, name = 'a') AS ue,
uniqExactIf(x, name IN ('a', 'b')) AS ui
FROM
FROM
(
SELECT
toNullable('a') AS name,

View File

@ -22,7 +22,7 @@ DROP TABLE installation_stats;
CREATE TEMPORARY TABLE Accounts (AccountID UInt64, Currency String);
SELECT AccountID
FROM
FROM
(
SELECT
AccountID,

View File

@ -12,14 +12,11 @@ SELECT
platform,
app
FROM test_00751.t_00751
WHERE (app =
(
WHERE (app = (
SELECT min(app)
FROM test_00751.u_00751
)) AND (platform =
(
SELECT
(
)) AND (platform = (
SELECT (
SELECT min(platform)
FROM test_00751.v_00751
)

View File

@ -10,7 +10,9 @@
[30000]
30000
[30000]
2016-06-15 23:01:04
['2016-06-15 23:01:04']
2016-06-15 23:01:04
['2016-06-15 23:01:04']
2016-06-15 23:00:16
['2016-06-15 23:00:16']
2016-06-15 23:00:16
['2016-06-15 23:00:16']
2016-04-02 17:23:12
['2016-04-02 17:23:12']

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS datetime;
CREATE TABLE datetime (d DateTime) ENGINE = Memory;
INSERT INTO datetime(d) VALUES(toDateTime('2016-06-15 23:00:00'));
CREATE TABLE datetime (d DateTime('UTC')) ENGINE = Memory;
INSERT INTO datetime(d) VALUES(toDateTime('2016-06-15 23:00:00', 'UTC'));
SELECT quantile(0.2)(d) FROM datetime;
SELECT quantiles(0.2)(d) FROM datetime;
@ -27,4 +27,7 @@ SELECT quantilesTDigest(0.2)(d) FROM datetime;
SELECT quantileTDigestWeighted(0.2)(d, 1) FROM datetime;
SELECT quantilesTDigestWeighted(0.2)(d, 1) FROM datetime;
SELECT quantileBFloat16(0.2)(d) FROM datetime;
SELECT quantilesBFloat16(0.2)(d) FROM datetime;
DROP TABLE datetime;

View File

@ -18,7 +18,7 @@
SELECT
n,
`finalizeAggregation(s)`
FROM
FROM
(
SELECT
n,

View File

@ -10,7 +10,7 @@ FROM t1
ALL INNER JOIN t2 ON b = t2.b
WHERE b = t2.b
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -21,7 +21,7 @@ FROM
ALL INNER JOIN t3 ON `--t1.a` = a
WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
b AS `--t1.b`,
@ -33,13 +33,13 @@ FROM
ALL INNER JOIN t3 ON `--t1.b` = b
WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.a`,
`--t2.a`,
a AS `--t3.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -52,14 +52,14 @@ FROM
ALL INNER JOIN t4 ON `--t1.a` = a
WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.b`,
`--t1.a`,
`--t2.b`,
b AS `--t3.b`
FROM
FROM
(
SELECT
b AS `--t1.b`,
@ -73,13 +73,13 @@ FROM
ALL INNER JOIN t4 ON `--t1.b` = b
WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.a`,
`--t2.a`,
a AS `--t3.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -92,13 +92,13 @@ FROM
ALL INNER JOIN t4 ON `--t2.a` = a
WHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.a`,
`--t2.a`,
a AS `--t3.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -111,13 +111,13 @@ FROM
ALL INNER JOIN t4 ON `--t3.a` = a
WHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.a`,
`--t2.a`,
a AS `--t3.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -130,13 +130,13 @@ FROM
ALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
WHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
`--t1.a`,
`--t2.a`,
a AS `--t3.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,
@ -149,10 +149,10 @@ FROM
ALL INNER JOIN t4 ON `--t3.a` = a
WHERE (`--t1.a` = `--t2.a`) AND (`--t2.a` = `--t3.a`) AND (`--t3.a` = a)
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT `--t1.a`
FROM
FROM
(
SELECT a AS `--t1.a`
FROM t1
@ -162,10 +162,10 @@ FROM
) AS `--.s`
CROSS JOIN t4
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT `--t1.a`
FROM
FROM
(
SELECT a AS `--t1.a`
FROM t1
@ -175,7 +175,7 @@ FROM
) AS `--.s`
CROSS JOIN t4
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT a AS `--t1.a`
FROM t1
@ -183,7 +183,7 @@ FROM
) AS `--.s`
CROSS JOIN t3
SELECT `--t1.a` AS `t1.a`
FROM
FROM
(
SELECT
a AS `--t1.a`,

View File

@ -1,7 +1,7 @@
SELECT
1 AS x,
x.y
FROM
FROM
(
SELECT 'Hello, world' AS y
) AS x

View File

@ -4,8 +4,7 @@ SELECT 1
SELECT 1
WHERE (1 IN (0, 2)) AND (2 = (identity(CAST(2, \'UInt8\')) AS subquery))
SELECT 1
WHERE 1 IN (
(
WHERE 1 IN ((
SELECT arrayJoin([1, 2, 3])
) AS subquery)
SELECT 1

View File

@ -48,7 +48,7 @@ dictGet('one_cell_cache_ints_overflow', 'i8', toUInt64(19)),
dictGet('one_cell_cache_ints_overflow', 'i8', toUInt64(20));
SELECT arrayMap(x -> dictGet('one_cell_cache_ints_overflow', 'i8', toUInt64(x)), array)
FROM
FROM
(
SELECT [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20] AS array
);

View File

@ -3,14 +3,14 @@ SELECT
v,
d,
i
FROM
FROM
(
SELECT
t.1 AS k,
t.2 AS v,
runningDifference(v) AS d,
runningDifference(cityHash64(t.1)) AS i
FROM
FROM
(
SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t
)
@ -26,14 +26,14 @@ SELECT
co2,
co3,
num
FROM
FROM
(
SELECT
co,
co2,
co3,
count() AS num
FROM
FROM
(
SELECT
1 AS co,
@ -51,13 +51,13 @@ WHERE (co != 0) AND (co2 != 2)
1 0 3 1
1 0 0 1
SELECT alias AS name
FROM
FROM
(
SELECT name AS alias
FROM system.settings
WHERE alias = \'enable_optimize_predicate_expression\'
)
ANY INNER JOIN
ANY INNER JOIN
(
SELECT name
FROM system.settings
@ -66,17 +66,17 @@ WHERE name = \'enable_optimize_predicate_expression\'
enable_optimize_predicate_expression
1 val11 val21 val31
SELECT ccc
FROM
FROM
(
SELECT 1 AS ccc
WHERE 0
UNION ALL
SELECT ccc
FROM
FROM
(
SELECT 2 AS ccc
)
ANY INNER JOIN
ANY INNER JOIN
(
SELECT 2 AS ccc
) USING (ccc)
@ -91,7 +91,7 @@ SELECT
b.ts,
b.id,
id_c
FROM
FROM
(
SELECT
ts,
@ -109,7 +109,7 @@ SELECT
b.ts AS `--b.ts`,
b.id AS `--b.id`,
id_c AS `--b.id_c`
FROM
FROM
(
SELECT
ts,
@ -129,7 +129,7 @@ WHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\')
2 3
4 5
SELECT dummy
FROM
FROM
(
SELECT dummy
FROM system.one
@ -141,13 +141,13 @@ SELECT
id,
value,
value_1
FROM
FROM
(
SELECT
1 AS id,
2 AS value
)
ALL INNER JOIN
ALL INNER JOIN
(
SELECT
1 AS id,

View File

@ -1,5 +1,5 @@
SELECT id
FROM
FROM
(
SELECT 1 AS id
UNION ALL
@ -20,7 +20,7 @@ ORDER BY id;
SELECT '---';
SELECT *
FROM
FROM
(
SELECT NULL AS x
) js1
@ -32,7 +32,7 @@ INNER JOIN
SELECT '---';
SELECT *
FROM
FROM
(
SELECT NULL AS x
) js1

View File

@ -3,7 +3,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT *
FROM default.test
@ -15,7 +15,7 @@ SELECT
id,
name,
value
FROM
FROM
(
SELECT *
FROM default.test
@ -23,7 +23,7 @@ FROM
) AS test_view
WHERE id = 2
SELECT id
FROM
FROM
(
SELECT *
FROM default.test
@ -31,7 +31,7 @@ FROM
) AS test_view
WHERE id = 1
SELECT id
FROM
FROM
(
SELECT *
FROM default.test

View File

@ -6,6 +6,6 @@ CREATE TABLE default.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'tes
CREATE TABLE default.distributed_tf\n(\n `n` Int8\n) AS cluster(\'test_shard_localhost\', \'default\', \'buffer\')
CREATE TABLE default.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+default.merge+format+CSV\', \'CSV\')
CREATE TABLE default.rich_syntax\n(\n `n` Int64\n) AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\')))
CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file
CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM\n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file
CREATE DICTIONARY default.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1))
16

View File

@ -1,5 +1,5 @@
SELECT number
FROM
FROM
(
SELECT zero AS number
FROM remote('127.0.0.2', system.zeros)

View File

@ -3,7 +3,7 @@ SELECT
sum(1 + n),
sum(n - 1),
sum(1 - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -13,7 +13,7 @@ SELECT
2 * sum(n),
sum(n) / 2,
sum(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -23,7 +23,7 @@ SELECT
1 + min(n),
min(n) - 1,
1 - min(n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -33,7 +33,7 @@ SELECT
2 * min(n),
min(n) / 2,
min(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -43,7 +43,7 @@ SELECT
1 + max(n),
max(n) - 1,
1 - max(n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -53,7 +53,7 @@ SELECT
2 * max(n),
max(n) / 2,
max(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -63,7 +63,7 @@ SELECT
sum(-1 + n),
sum(n - -1),
sum(-1 - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -73,7 +73,7 @@ SELECT
-2 * sum(n),
sum(n) / -2,
sum(-1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -83,7 +83,7 @@ SELECT
-1 + min(n),
min(n) - -1,
-1 - min(n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -93,7 +93,7 @@ SELECT
-2 * max(n),
max(n) / -2,
min(-1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -103,7 +103,7 @@ SELECT
-1 + max(n),
max(n) - -1,
-1 - max(n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -113,7 +113,7 @@ SELECT
-2 * min(n),
min(n) / -2,
max(-1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -123,7 +123,7 @@ SELECT
sum(abs(2) + n),
sum(n - abs(2)),
sum(1 - abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -133,7 +133,7 @@ SELECT
sum(abs(2) * n),
sum(n / abs(2)),
sum(1 / abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -143,7 +143,7 @@ SELECT
min(abs(2) + n),
min(n - abs(2)),
1 - min(abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -153,7 +153,7 @@ SELECT
min(abs(2) * n),
min(n / abs(2)),
min(1 / abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -163,7 +163,7 @@ SELECT
max(abs(2) + n),
max(n - abs(2)),
1 - max(abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -173,7 +173,7 @@ SELECT
max(abs(2) * n),
max(n / abs(2)),
max(1 / abs(2))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -183,7 +183,7 @@ SELECT
sum(abs(n) + n),
sum(n - abs(n)),
sum(1 - abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -193,7 +193,7 @@ SELECT
sum(abs(n) * n),
sum(n / abs(n)),
sum(1 / abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -203,7 +203,7 @@ SELECT
min(abs(n) + n),
min(n - abs(n)),
1 - min(abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -213,7 +213,7 @@ SELECT
min(abs(n) * n),
min(n / abs(n)),
min(1 / abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -223,7 +223,7 @@ SELECT
max(abs(n) + n),
max(n - abs(n)),
1 - max(abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -233,7 +233,7 @@ SELECT
max(abs(n) * n),
max(n / abs(n)),
max(1 / abs(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -243,7 +243,7 @@ SELECT
sum(1 + (n * n)),
sum((n * n) - 1),
sum(1 - (n * n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -253,7 +253,7 @@ SELECT
sum((2 * n) * n),
sum(n * n) / 2,
sum((1 / n) * n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -263,7 +263,7 @@ SELECT
1 + min(n * n),
min(n * n) - 1,
1 - min(n * n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -273,7 +273,7 @@ SELECT
min((2 * n) * n),
min(n * n) / 2,
min((1 / n) * n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -283,7 +283,7 @@ SELECT
1 + max(n * n),
max(n * n) - 1,
1 - max(n * n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -293,7 +293,7 @@ SELECT
max((2 * n) * n),
max(n * n) / 2,
max((1 / n) * n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -303,7 +303,7 @@ SELECT
sum((1 + 1) + n),
sum((1 + n) - 1),
sum((1 + 1) - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -313,7 +313,7 @@ SELECT
sum(1 + (2 * n)),
sum(1 + (n / 2)),
sum(1 + (1 / n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -323,7 +323,7 @@ SELECT
min((1 + 1) + n),
(1 + min(n)) - 1,
min((1 + 1) - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -333,7 +333,7 @@ SELECT
1 + min(2 * n),
1 + min(n / 2),
1 + min(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -343,7 +343,7 @@ SELECT
max((1 + 1) + n),
(1 + max(n)) - 1,
max((1 + 1) - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -353,7 +353,7 @@ SELECT
1 + max(2 * n),
1 + max(n / 2),
1 + max(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -363,7 +363,7 @@ SELECT
sum((-1 + n) + -1),
sum((n - -1) + -1),
sum((-1 - n) + -1)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -373,7 +373,7 @@ SELECT
(-2 * sum(n)) * -1,
(sum(n) / -2) / -1,
sum(-1 / n) / -1
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -383,7 +383,7 @@ SELECT
(-1 + min(n)) + -1,
(min(n) - -1) + -1,
(-1 - min(n)) + -1
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -393,7 +393,7 @@ SELECT
(-2 * min(n)) * -1,
(min(n) / -2) / -1,
max(-1 / n) / -1
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -403,7 +403,7 @@ SELECT
(-1 + max(n)) + -1,
(max(n) - -1) + -1,
(-1 - max(n)) + -1
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
@ -413,43 +413,43 @@ SELECT
(-2 * max(n)) * -1,
(max(n) / -2) / -1,
min(-1 / n) / -1
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT ((sum(n + 1) + sum(1 + n)) + sum(n - 1)) + sum(1 - n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT (((sum(n) * 2) + (2 * sum(n))) + (sum(n) / 2)) + sum(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT (((min(n) + 1) + (1 + min(n))) + (min(n) - 1)) + (1 - min(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT (((min(n) * 2) + (2 * min(n))) + (min(n) / 2)) + min(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT (((max(n) + 1) + (1 + max(n))) + (max(n) - 1)) + (1 - max(n))
FROM
FROM
(
SELECT number AS n
FROM numbers(10)
)
SELECT (((max(n) * 2) + (2 * max(n))) + (max(n) / 2)) + max(1 / n)
FROM
FROM
(
SELECT number AS n
FROM numbers(10)

View File

@ -1,8 +1,8 @@
SELECT number
FROM
FROM
(
SELECT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(3)
@ -13,10 +13,10 @@ ORDER BY number ASC
1
2
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(3)
@ -29,10 +29,10 @@ ORDER BY number ASC
1
2
SELECT number
FROM
FROM
(
SELECT number
FROM
FROM
(
SELECT DISTINCT number % 2 AS number
FROM numbers(3)
@ -42,10 +42,10 @@ ORDER BY number ASC
0
1
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number % 2 AS number
FROM numbers(3)

View File

@ -74,7 +74,7 @@ GROUP BY
number % 5
ORDER BY a ASC
SELECT foo
FROM
FROM
(
SELECT number AS foo
FROM numbers(1)
@ -155,7 +155,7 @@ GROUP BY
number % 5
ORDER BY a ASC
SELECT foo
FROM
FROM
(
SELECT anyLast(number) AS foo
FROM numbers(1)

View File

@ -16,21 +16,21 @@
2 2
3 3
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
ORDER BY x ASC
)
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
ORDER BY x ASC
)
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
@ -43,7 +43,7 @@ SELECT
a,
b,
c
FROM
FROM
(
SELECT number + 2 AS key
FROM numbers(4)
@ -84,7 +84,7 @@ ORDER BY
2 2
3 3
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
@ -93,7 +93,7 @@ FROM
exp(x) ASC
)
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
@ -102,7 +102,7 @@ FROM
exp(exp(x)) ASC
)
SELECT groupArray(x)
FROM
FROM
(
SELECT number AS x
FROM numbers(3)
@ -115,7 +115,7 @@ SELECT
a,
b,
c
FROM
FROM
(
SELECT number + 2 AS key
FROM numbers(4)

View File

@ -1,7 +1,7 @@
SELECT
k,
groupArrayMovingSum(v)
FROM
FROM
(
SELECT
k,

View File

@ -4,7 +4,7 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -15,7 +15,7 @@ SELECT
uniqHLL12(x + y),
uniqCombined(x + y),
uniqCombined64(x + y)
FROM
FROM
(
SELECT
number % 2 AS x,
@ -28,7 +28,7 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -39,7 +39,7 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -50,7 +50,7 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -61,13 +61,13 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
)
SELECT uniqExact(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -80,7 +80,7 @@ SELECT
uniqHLL12(x),
uniqCombined(x),
uniqCombined64(x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -91,7 +91,7 @@ SELECT
uniqHLL12(x + y),
uniqCombined(x + y),
uniqCombined64(x + y)
FROM
FROM
(
SELECT
number % 2 AS x,
@ -104,7 +104,7 @@ SELECT
uniqHLL12(-x),
uniqCombined(-x),
uniqCombined64(-x)
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -115,7 +115,7 @@ SELECT
uniqHLL12(bitNot(x)),
uniqCombined(bitNot(x)),
uniqCombined64(bitNot(x))
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -126,7 +126,7 @@ SELECT
uniqHLL12(bitNot(-x)),
uniqCombined(bitNot(-x)),
uniqCombined64(bitNot(-x))
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
@ -137,13 +137,13 @@ SELECT
uniqHLL12(-bitNot(-x)),
uniqCombined(-bitNot(-x)),
uniqCombined64(-bitNot(-x))
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)
)
SELECT uniqExact(-bitNot(-x))
FROM
FROM
(
SELECT number % 2 AS x
FROM numbers(10)

View File

@ -1,4 +1,5 @@
1
1
Code: 516
1
Code: 516

View File

@ -4,6 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'EmptyHeader;' -d 'SELECT 1'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-User: default' -d 'SELECT 1'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-User: header_test' -d 'SELECT 1' | grep -o 'Code: 516'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Key: ' -d 'SELECT 1'

View File

@ -11,7 +11,7 @@ insert into tracking_events_tmp select 2, '2020-07-10' from numbers(1881);
insert into tracking_events_tmp select 2, '2020-07-11' from numbers(1623);
SELECT EventDate
FROM
FROM
(
SELECT EventDate
FROM tracking_events_tmp AS t1

View File

@ -3,7 +3,7 @@
SELECT
number,
square_number
FROM
FROM
(
WITH number * 2 AS square_number
SELECT

View File

@ -1,13 +1,13 @@
SELECT DISTINCT number
FROM numbers(1)
SELECT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(1)
)
SELECT DISTINCT number * 2
FROM
FROM
(
SELECT DISTINCT
number * 2,
@ -15,7 +15,7 @@ FROM
FROM numbers(1)
)
SELECT number
FROM
FROM
(
SELECT DISTINCT number * 2 AS number
FROM numbers(1)
@ -23,7 +23,7 @@ FROM
SELECT
b,
a
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -31,7 +31,7 @@ FROM
FROM numbers(100)
)
SELECT DISTINCT a
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -39,10 +39,10 @@ FROM
FROM numbers(100)
)
SELECT a
FROM
FROM
(
SELECT DISTINCT a
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -51,12 +51,12 @@ FROM
)
)
SELECT DISTINCT a
FROM
FROM
(
SELECT
a,
b
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -67,12 +67,12 @@ FROM
SELECT
a,
b
FROM
FROM
(
SELECT
b,
a
FROM
FROM
(
SELECT DISTINCT
number AS a,
@ -83,13 +83,13 @@ FROM
SELECT
a,
b
FROM
FROM
(
SELECT
b,
a,
a + b
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -98,10 +98,10 @@ FROM
)
)
SELECT DISTINCT a
FROM
FROM
(
SELECT a
FROM
FROM
(
SELECT DISTINCT
number % 2 AS a,
@ -110,21 +110,21 @@ FROM
)
)
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(1)
) AS t1
CROSS JOIN numbers(2) AS t2
SELECT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(1) AS t1
CROSS JOIN numbers(2) AS t2
)
SELECT DISTINCT number
FROM
FROM
(
SELECT DISTINCT number
FROM numbers(1)

View File

@ -1,5 +1,5 @@
0 0
WITH it AS
WITH it AS
(
SELECT *
FROM numbers(1)
@ -7,4 +7,5 @@ WITH it AS
SELECT
number,
number
FROM it AS i
FROM
it AS i

View File

@ -13,7 +13,7 @@ WITH 1 AS x
SELECT x
WITH 1 AS x
SELECT x
FROM
FROM
(
WITH 1 AS x
SELECT x
@ -22,7 +22,7 @@ WITH 1 AS x
SELECT
y,
x
FROM
FROM
(
WITH 2 AS x
SELECT 2 AS y
@ -39,7 +39,7 @@ WITH 2 AS x
SELECT x
WITH
5 AS q1,
x AS
x AS
(
WITH 5 AS q1
SELECT
@ -51,4 +51,5 @@ WITH
SELECT
b,
a
FROM x
FROM
x

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