Merge branch 'aes-fix-strange-code' of github.com:yandex/ClickHouse into aes-fix-strange-code

This commit is contained in:
Alexey Milovidov 2021-01-23 12:14:30 +03:00
commit 0f3b9f53b5
46 changed files with 255 additions and 103 deletions

View File

@ -229,8 +229,12 @@ public:
inline UInt8 daysInMonth(UInt16 year, UInt8 month) const
{
UInt16 idx = year - DATE_LUT_MIN_YEAR;
if (unlikely(idx >= DATE_LUT_YEARS))
return 31; /// Implementation specific behaviour on overflow.
/// 32 makes arithmetic more simple.
DayNum any_day_of_month = DayNum(years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1));
DayNum any_day_of_month = DayNum(years_lut[idx] + 32 * (month - 1));
return lut[any_day_of_month].days_in_month;
}

View File

@ -515,7 +515,7 @@ Example:
## JSONAsString {#jsonasstring}
In this format, a single JSON object is interpreted as a single value. If input has several JSON objects (comma separated) they will be interpreted as a sepatate rows.
In this format, a single JSON object is interpreted as a single value. If the input has several JSON objects (comma separated) they will be interpreted as separate rows.
This format can only be parsed for table with a single field of type [String](../sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](../sql-reference/statements/create/table.md#default) or [MATERIALIZED](../sql-reference/statements/create/table.md#materialized), or omitted. Once you collect whole JSON object to string you can use [JSON functions](../sql-reference/functions/json-functions.md) to process it.
@ -526,7 +526,7 @@ Query:
``` sql
DROP TABLE IF EXISTS json_as_string;
CREATE TABLE json_as_string (json String) ENGINE = Memory;
INSERT INTO json_as_string FORMAT JSONAsString {"foo":{"bar":{"x":"y"},"baz":1}},{},{"any json stucture":1}
INSERT INTO json_as_string (json) FORMAT JSONAsString {"foo":{"bar":{"x":"y"},"baz":1}},{},{"any json stucture":1}
SELECT * FROM json_as_string;
```
@ -540,7 +540,6 @@ Result:
└───────────────────────────────────┘
```
## JSONCompact {#jsoncompact}
## JSONCompactString {#jsoncompactstring}

View File

@ -71,8 +71,8 @@ Parameters:
<remote_servers>
<source_cluster>
<!--
source cluster & destination clusters accepts exactly the same
parameters as parameters for usual Distributed table
source cluster & destination clusters accept exactly the same
parameters as parameters for the usual Distributed table
see https://clickhouse.tech/docs/en/engines/table-engines/special/distributed/
-->
<shard>

View File

@ -24,6 +24,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
| [Vertical](#vertical) | ✗ | ✔ |
| [VerticalRaw](#verticalraw) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONString](#jsonstring) | ✗ | ✔ |
| [JSONCompact](#jsoncompact) | ✗ | ✔ |
| [JSONCompactString](#jsoncompactstring) | ✗ | ✔ |
@ -490,6 +491,33 @@ ClickHouse поддерживает [NULL](../sql-reference/syntax.md), кото
}
```
## JSONAsString {#jsonasstring}
В этом формате один объект JSON интерпретируется как одно строковое значение. Если входные данные имеют несколько объектов JSON, разделенных запятой, то они будут интерпретироваться как отдельные строки таблицы.
В этом формате парситься может только таблица с единственным полем типа [String](../sql-reference/data-types/string.md). Остальные столбцы должны быть заданы как `DEFAULT` или `MATERIALIZED`(смотрите раздел [Значения по умолчанию](../sql-reference/statements/create/table.md#create-default-values)), либо отсутствовать. Для дальнейшей обработки объекта JSON, представленного в строке, вы можете использовать [функции для работы с JSON](../sql-reference/functions/json-functions.md).
**Пример**
Запрос:
``` sql
DROP TABLE IF EXISTS json_as_string;
CREATE TABLE json_as_string (json String) ENGINE = Memory;
INSERT INTO json_as_string (json) FORMAT JSONAsString {"foo":{"bar":{"x":"y"},"baz":1}},{},{"any json stucture":1}
SELECT * FROM json_as_string;
```
Результат:
``` text
┌─json──────────────────────────────┐
│ {"foo":{"bar":{"x":"y"},"baz":1}} │
│ {} │
│ {"any json stucture":1} │
└───────────────────────────────────┘
```
## JSONCompact {#jsoncompact}
## JSONCompactString {#jsoncompactstring}

View File

@ -67,11 +67,21 @@ $ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --bas
<!-- Configuration of clusters as in an ordinary server config -->
<remote_servers>
<source_cluster>
<!--
source cluster & destination clusters accept exactly the same
parameters as parameters for the usual Distributed table
see https://clickhouse.tech/docs/ru/engines/table-engines/special/distributed/
-->
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
<!--
<user>default</user>
<password>default</password>
<secure>1</secure>
-->
</replica>
</shard>
...

View File

@ -479,7 +479,7 @@ void QueryFuzzer::addTableLike(const ASTPtr ast)
{
if (table_like_map.size() > 1000)
{
return;
table_like_map.clear();
}
const auto name = ast->formatForErrorMessage();
@ -493,7 +493,7 @@ void QueryFuzzer::addColumnLike(const ASTPtr ast)
{
if (column_like_map.size() > 1000)
{
return;
column_like_map.clear();
}
const auto name = ast->formatForErrorMessage();
@ -507,10 +507,12 @@ void QueryFuzzer::collectFuzzInfoRecurse(const ASTPtr ast)
{
if (auto * impl = dynamic_cast<ASTWithAlias *>(ast.get()))
{
if (aliases_set.size() < 1000)
if (aliases_set.size() > 1000)
{
aliases_set.insert(impl->alias);
aliases_set.clear();
}
aliases_set.insert(impl->alias);
}
if (typeid_cast<ASTLiteral *>(ast.get()))

View File

@ -28,7 +28,7 @@ public:
using ValueT = MaxFieldType<Value, Weight>;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
void NO_SANITIZE_UNDEFINED add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
const auto& weights = static_cast<const DecimalOrVectorCol<Weight> &>(*columns[1]);

View File

@ -54,6 +54,7 @@ namespace ErrorCodes
extern const int UNKNOWN_PACKET_FROM_SERVER;
extern const int SUPPORT_IS_DISABLED;
extern const int BAD_ARGUMENTS;
extern const int EMPTY_DATA_PASSED;
}
@ -545,6 +546,9 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String
{
/// NOTE 'Throttler' is not used in this method (could use, but it's not important right now).
if (input.eof())
throw Exception("Buffer is empty (some kind of corruption)", ErrorCodes::EMPTY_DATA_PASSED);
writeVarUInt(Protocol::Client::Data, *out);
writeStringBinary(name, *out);

View File

@ -188,7 +188,10 @@ struct ConvertImpl
(std::is_same_v<FromFieldType, UInt128> || std::is_same_v<ToFieldType, UInt128>))
{
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
{
vec_to[i] = 0;
(*vec_null_map_to)[i] = true;
}
else
throw Exception("Unexpected UInt128 to big int conversion", ErrorCodes::NOT_IMPLEMENTED);
}
@ -215,8 +218,11 @@ struct ConvertImpl
if (convert_result)
vec_to[i] = result;
else
{
vec_to[i] = static_cast<ToFieldType>(0);
(*vec_null_map_to)[i] = true;
}
}
else
{
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
@ -238,6 +244,7 @@ struct ConvertImpl
{
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
{
vec_to[i] = 0;
(*vec_null_map_to)[i] = true;
continue;
}
@ -255,6 +262,7 @@ struct ConvertImpl
{
if (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
{
vec_to[i] = 0;
(*vec_null_map_to)[i] = true;
}
else

View File

@ -119,7 +119,7 @@ public:
if (offset > 0)
sink = GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, static_cast<size_t>(offset - 1));
else
sink = GatherUtils::sliceFromRightConstantOffsetUnbounded(*source, static_cast<size_t>(-offset));
sink = GatherUtils::sliceFromRightConstantOffsetUnbounded(*source, -static_cast<size_t>(offset));
}
else if (isColumnConst(*length_column))
{
@ -127,7 +127,7 @@ public:
if (offset > 0)
sink = GatherUtils::sliceFromLeftConstantOffsetBounded(*source, static_cast<size_t>(offset - 1), length);
else
sink = GatherUtils::sliceFromRightConstantOffsetBounded(*source, static_cast<size_t>(-offset), length);
sink = GatherUtils::sliceFromRightConstantOffsetBounded(*source, -static_cast<size_t>(offset), length);
}
else
sink = GatherUtils::sliceDynamicOffsetBounded(*source, *offset_column, *length_column);

View File

@ -99,6 +99,12 @@ void geodistInit()
}
}
inline NO_SANITIZE_UNDEFINED size_t floatToIndex(float x)
{
/// Implementation specific behaviour on overflow or infinite value.
return static_cast<size_t>(x);
}
inline float geodistDegDiff(float f)
{
f = fabsf(f);
@ -110,7 +116,7 @@ inline float geodistDegDiff(float f)
inline float geodistFastCos(float x)
{
float y = fabsf(x) * (COS_LUT_SIZE / PI / 2);
size_t i = static_cast<size_t>(y);
size_t i = floatToIndex(y);
y -= i;
i &= (COS_LUT_SIZE - 1);
return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y;
@ -119,7 +125,7 @@ inline float geodistFastCos(float x)
inline float geodistFastSin(float x)
{
float y = fabsf(x) * (COS_LUT_SIZE / PI / 2);
size_t i = static_cast<size_t>(y);
size_t i = floatToIndex(y);
y -= i;
i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2
return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y;
@ -139,7 +145,7 @@ inline float geodistFastAsinSqrt(float x)
{
// distance under 17083 km, 512-entry LUT error under 0.00072%
x *= ASIN_SQRT_LUT_SIZE;
size_t i = static_cast<size_t>(x);
size_t i = floatToIndex(x);
return asin_sqrt_lut[i] + (asin_sqrt_lut[i + 1] - asin_sqrt_lut[i]) * (x - i);
}
return asinf(sqrtf(x)); // distance over 17083 km, just compute exact
@ -177,7 +183,7 @@ float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg)
/// But if longitude is close but latitude is different enough, there is no difference between meridian and great circle line.
float latitude_midpoint = (lat1deg + lat2deg + 180) * METRIC_LUT_SIZE / 360; // [-90, 90] degrees -> [0, KTABLE] indexes
size_t latitude_midpoint_index = static_cast<size_t>(latitude_midpoint) & (METRIC_LUT_SIZE - 1);
size_t latitude_midpoint_index = floatToIndex(latitude_midpoint) & (METRIC_LUT_SIZE - 1);
/// This is linear interpolation between two table items at index "latitude_midpoint_index" and "latitude_midpoint_index + 1".

View File

@ -46,7 +46,7 @@ private:
/// It is possible to track value from previous columns, to calculate continuously across all columnss. Not implemented.
template <typename Src, typename Dst>
static void process(const PaddedPODArray<Src> & src, PaddedPODArray<Dst> & dst, const NullMap * null_map)
static NO_SANITIZE_UNDEFINED void process(const PaddedPODArray<Src> & src, PaddedPODArray<Dst> & dst, const NullMap * null_map)
{
size_t size = src.size();
dst.resize(size);
@ -76,6 +76,7 @@ private:
else
{
auto cur = src[i];
/// Overflow is Ok.
dst[i] = static_cast<Dst>(cur) - prev;
prev = cur;
}

View File

@ -91,9 +91,11 @@ public:
if (column_start_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1);
sliceFromLeftConstantOffsetUnbounded(
source, StringSink(*col_res, input_rows_count), static_cast<size_t>(start_value - 1));
else if (start_value < 0)
sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value);
sliceFromRightConstantOffsetUnbounded(
source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value));
else
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
}
@ -105,9 +107,11 @@ public:
if (column_start_const && column_length_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value);
sliceFromLeftConstantOffsetBounded(
source, StringSink(*col_res, input_rows_count), static_cast<size_t>(start_value - 1), length_value);
else if (start_value < 0)
sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value);
sliceFromRightConstantOffsetBounded(
source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value), length_value);
else
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
}

View File

@ -120,7 +120,7 @@ namespace
session->setProxyHost(proxy_host);
session->setProxyPort(proxy_port);
#if !defined(ARCADIA_BUILD)
#if !defined(ARCADIA_BUILD) && defined(POCO_CLICKHOUSE_PATCH)
session->setProxyProtocol(proxy_scheme);
/// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS.

View File

@ -62,6 +62,8 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const
void ColumnDescription::writeText(WriteBuffer & buf) const
{
/// NOTE: Serialization format is insane.
writeBackQuotedString(name, buf);
writeChar(' ', buf);
writeEscapedString(type->getName(), buf);
@ -99,15 +101,23 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
void ColumnDescription::readText(ReadBuffer & buf)
{
readBackQuotedString(name, buf);
assertChar(' ', buf);
String type_string;
readEscapedString(type_string, buf);
type = DataTypeFactory::instance().get(type_string);
if (checkChar('\t', buf))
{
String modifiers;
readEscapedStringUntilEOL(modifiers, buf);
ParserColumnDeclaration column_parser(/* require type */ true);
String column_line;
readEscapedStringUntilEOL(column_line, buf);
ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
ASTPtr ast = parseQuery(column_parser, "x T " + modifiers, "column parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
if (const auto * col_ast = ast->as<ASTColumnDeclaration>())
{
name = col_ast->name;
type = DataTypeFactory::instance().get(col_ast->type);
if (col_ast->default_expression)
{
default_desc.kind = columnDefaultKindFromString(col_ast->default_specifier);
@ -125,6 +135,7 @@ void ColumnDescription::readText(ReadBuffer & buf)
}
else
throw Exception("Cannot parse column description", ErrorCodes::CANNOT_PARSE_TEXT);
}
}

View File

@ -46,6 +46,7 @@ namespace ErrorCodes
extern const int CHECKSUM_DOESNT_MATCH;
extern const int TOO_LARGE_SIZE_COMPRESSED;
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int EMPTY_DATA_PASSED;
}
@ -170,6 +171,7 @@ namespace
bool isFileBrokenErrorCode(int code, bool remote_error)
{
return code == ErrorCodes::CHECKSUM_DOESNT_MATCH
|| code == ErrorCodes::EMPTY_DATA_PASSED
|| code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|| code == ErrorCodes::UNKNOWN_CODEC

View File

@ -717,7 +717,7 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size
}
#if defined(__SSE2__) && !defined(MEMORY_SANITIZER)
// Could happen only if string is not padded with zeroes, and we accidentally hopped over end of data.
// Could happen only if string is not padded with zeros, and we accidentally hopped over the end of data.
if (*token_start > len)
return false;
*token_len = std::min(len - *token_start, *token_len);

View File

@ -45,10 +45,19 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n")
return full_path
def run_and_check(args, env=None, shell=False):
res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell)
if res.returncode != 0:
# check_call(...) from subprocess does not print stderr, so we do it manually
print('Stderr:\n{}\n'.format(res.stderr))
print('Stdout:\n{}\n'.format(res.stdout))
raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr))
def subprocess_check_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.check_call(args)
run_and_check(args)
def subprocess_call(args):
@ -56,7 +65,6 @@ def subprocess_call(args):
# print('run:', ' ' . join(args))
subprocess.call(args)
def get_odbc_bridge_path():
path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH')
if path is None:
@ -390,9 +398,9 @@ class ClickHouseCluster:
raise Exception("You should specity ipv4_address in add_node method")
self._replace(node.docker_compose_path, node.ipv4_address, new_ip)
node.ipv4_address = new_ip
subprocess.check_call(self.base_cmd + ["stop", node.name])
subprocess.check_call(self.base_cmd + ["rm", "--force", "--stop", node.name])
subprocess.check_call(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
run_and_check(self.base_cmd + ["stop", node.name])
run_and_check(self.base_cmd + ["rm", "--force", "--stop", node.name])
run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
node.ip_address = self.get_instance_ip(node.name)
node.client = Client(node.ip_address, command=self.client_bin_path)
start_deadline = time.time() + 20.0 # seconds
@ -637,7 +645,7 @@ class ClickHouseCluster:
os.mkdir(zk_log_data_path)
env['ZK_DATA' + str(i)] = zk_data_path
env['ZK_DATA_LOG' + str(i)] = zk_log_data_path
subprocess.check_call(self.base_zookeeper_cmd + common_opts, env=env)
run_and_check(self.base_zookeeper_cmd + common_opts, env=env)
for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120)
@ -662,7 +670,7 @@ class ClickHouseCluster:
print('Setup kerberized kafka')
env = os.environ.copy()
env['KERBERIZED_KAFKA_DIR'] = instance.path + '/'
subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env)
run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env)
self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1')
if self.with_rabbitmq and self.base_rabbitmq_cmd:
subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes'])
@ -678,13 +686,13 @@ class ClickHouseCluster:
print('Setup kerberized HDFS')
env = os.environ.copy()
env['KERBERIZED_HDFS_DIR'] = instance.path + '/'
subprocess.check_call(self.base_kerberized_hdfs_cmd + common_opts, env=env)
run_and_check(self.base_kerberized_hdfs_cmd + common_opts, env=env)
self.make_hdfs_api(kerberized=True)
self.wait_hdfs_to_start(timeout=300)
if self.with_mongo and self.base_mongo_cmd:
print('Setup Mongo')
subprocess_check_call(self.base_mongo_cmd + common_opts)
run_and_check(self.base_mongo_cmd + common_opts)
self.wait_mongo_to_start(30)
if self.with_redis and self.base_redis_cmd:
@ -709,7 +717,7 @@ class ClickHouseCluster:
minio_start_cmd = self.base_minio_cmd + common_opts
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
subprocess.check_call(minio_start_cmd, env=env)
run_and_check(minio_start_cmd, env=env)
try:
logging.info("Trying to connect to Minio...")
@ -754,7 +762,7 @@ class ClickHouseCluster:
sanitizer_assert_instance = None
with open(self.docker_logs_path, "w+") as f:
try:
subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
except Exception as e:
print("Unable to get logs from docker.")
f.seek(0)

View File

@ -191,7 +191,7 @@ class _NetworkManager:
# before running
for i in range(5):
try:
subprocess.check_call("docker pull yandex/clickhouse-integration-helper", shell=True)
subprocess.check_call("docker pull yandex/clickhouse-integration-helper", shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
break
except:
time.sleep(i)

View File

@ -1,10 +1,9 @@
import os
import pytest
import subprocess
import sys
import time
import grpc
from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseCluster, run_and_check
from threading import Thread
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@ -15,7 +14,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
proto_dir = os.path.join(SCRIPT_DIR, './protos')
gen_dir = os.path.join(SCRIPT_DIR, './_gen')
os.makedirs(gen_dir, exist_ok=True)
subprocess.check_call(
run_and_check(
'python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
{proto_dir}/clickhouse_grpc.proto'.format(proto_dir=proto_dir, gen_dir=gen_dir), shell=True)

View File

@ -1,9 +1,8 @@
import os
import pytest
import subprocess
import sys
import grpc
from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseCluster, run_and_check
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@ -13,7 +12,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
proto_dir = os.path.join(SCRIPT_DIR, './protos')
gen_dir = os.path.join(SCRIPT_DIR, './_gen')
os.makedirs(gen_dir, exist_ok=True)
subprocess.check_call(
run_and_check(
'python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
{proto_dir}/clickhouse_grpc.proto'.format(proto_dir=proto_dir, gen_dir=gen_dir), shell=True)

View File

@ -1,13 +1,11 @@
import time
import pymysql.cursors
import subprocess
import pytest
from helpers.client import QueryRuntimeException
from helpers.network import PartitionManager
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import get_docker_compose_path
from helpers.cluster import get_docker_compose_path, run_and_check
import random
import threading
@ -684,7 +682,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name):
t = threading.Thread(target=insert, args=(10000,))
t.start()
subprocess.check_call(
run_and_check(
['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'stop'])
finally:
with pytest.raises(QueryRuntimeException) as execption:
@ -692,7 +690,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name):
clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test")
assert "Master maybe lost." in str(execption.value)
subprocess.check_call(
run_and_check(
['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'start'])
mysql_node.wait_mysql_to_start(120)

View File

@ -1,12 +1,11 @@
import os
import os.path as p
import subprocess
import time
import pwd
import re
import pymysql.cursors
import pytest
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
import docker
from . import materialize_with_ddl
@ -87,7 +86,7 @@ class MySQLNodeInstance:
print("Can't connect to MySQL " + str(ex))
time.sleep(0.5)
subprocess.check_call(['docker-compose', 'ps', '--services', 'all'])
run_and_check(['docker-compose', 'ps', '--services', 'all'])
raise Exception("Cannot wait MySQL container")
@pytest.fixture(scope="module")
@ -96,13 +95,13 @@ def started_mysql_5_7():
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose)
try:
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
mysql_node.wait_mysql_to_start(120)
yield mysql_node
finally:
mysql_node.close()
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
'--remove-orphans'])
@ -112,13 +111,13 @@ def started_mysql_8_0():
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose)
try:
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
mysql_node.wait_mysql_to_start(120)
yield mysql_node
finally:
mysql_node.close()
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
'--remove-orphans'])

View File

@ -3,14 +3,13 @@
import datetime
import math
import os
import subprocess
import time
import docker
import pymysql.connections
import pytest
from docker.models.containers import Container
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
DOCKER_COMPOSE_PATH = get_docker_compose_path()
@ -35,7 +34,7 @@ def server_address():
@pytest.fixture(scope='module')
def mysql_client():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1')
@ -62,7 +61,7 @@ def mysql_server(mysql_client):
@pytest.fixture(scope='module')
def golang_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_golang1_1')
@ -70,7 +69,7 @@ def golang_container():
@pytest.fixture(scope='module')
def php_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_php1_1')
@ -78,7 +77,7 @@ def php_container():
@pytest.fixture(scope='module')
def nodejs_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1')
@ -86,7 +85,7 @@ def nodejs_container():
@pytest.fixture(scope='module')
def java_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_java1_1')

View File

@ -1,11 +1,8 @@
# -*- coding: utf-8 -*-
import datetime
import decimal
import os
import subprocess
import sys
import time
import uuid
@ -14,7 +11,7 @@ import docker
import psycopg2 as py_psql
import psycopg2.extras
import pytest
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
psycopg2.extras.register_uuid()
@ -42,7 +39,7 @@ def server_address():
@pytest.fixture(scope='module')
def psql_client():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
yield docker.from_env().containers.get(cluster.project_name + '_psql_1')
@ -66,7 +63,7 @@ def psql_server(psql_client):
@pytest.fixture(scope='module')
def java_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml')
subprocess.check_call(
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
yield docker.from_env().containers.get(cluster.project_name + '_java_1')

View File

@ -5,19 +5,9 @@ fun:__gxx_personality_*
# it is OK. Reproduce with "select ngramDistanceCaseInsensitive(materialize(''), '')"
fun:tolower
# May be it's not OK, but suppress it to run other tests
# Some functions in OpenSSL:
fun:probable_prime
fun:BN_bin2bn
fun:BN_add_word
fun:bn_div_fixed_top
fun:bn_mul_words
fun:BN_cmp
# Suppress some failures in contrib so that we can enable MSan in CI.
# Ideally, we should report these upstream.
src:*/contrib/zlib-ng/*
src:*/contrib/openssl/*
src:*/contrib/simdjson/*
src:*/contrib/lz4/*

View File

@ -0,0 +1,15 @@
<test max_ignored_relative_change="0.2">
<settings>
<max_memory_usage>15G</max_memory_usage>
</settings>
<create_query>CREATE TABLE t (x UInt64) ENGINE = Memory</create_query>
<!-- use less threads to save memory -->
<fill_query>INSERT INTO t SELECT number AS x FROM numbers_mt(200000000) SETTINGS max_threads = 8</fill_query>
<drop_query>DROP TABLE IF EXISTS t</drop_query>
<query>SELECT cast(x, 'Int64'), cast(x, 'Int128'), cast(x, 'Int256'), cast(x, 'String'), cast(x, 'Decimal64(8)') FROM t FORMAT Null</query>
<query>SELECT accurateCast(x, 'Int64'), accurateCast(x, 'Int128'), accurateCast(x, 'Int256'), accurateCast(x, 'String'), accurateCast(x, 'Decimal64(8)') FROM t FORMAT Null</query>
<query>SELECT accurateCastOrNull(x, 'Int64'), accurateCastOrNull(x, 'Int128'), accurateCastOrNull(x, 'Int256'), accurateCastOrNull(x, 'String'), accurateCastOrNull(x, 'Decimal64(8)') FROM t FORMAT Null</query>
</test>

View File

@ -28,3 +28,5 @@
\N
\N
\N
127
\N

View File

@ -33,3 +33,5 @@ SELECT accurateCastOrNull(inf, 'UInt64');
SELECT accurateCastOrNull(inf, 'UInt256');
SELECT accurateCastOrNull(nan, 'UInt64');
SELECT accurateCastOrNull(nan, 'UInt256');
SELECT accurateCastOrNull(number + 127, 'Int8') AS x FROM numbers (2) ORDER BY x;

View File

@ -0,0 +1 @@
[0,0,0]

View File

@ -0,0 +1 @@
SELECT arraySlice(groupArray(x), -9223372036854775808, NULL) AS y FROM (SELECT '6553.5', uniqState(NULL) AS x FROM numbers(3) GROUP BY number);

View File

@ -0,0 +1 @@
SELECT k, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM (SELECT arrayJoin([(NULL, 65535), ('a', 7), ('a', 3), ('b', 11), ('b', 2), ('', -9223372036854775808)]) AS t)) WHERE i = 9223372036854775807;

View File

@ -0,0 +1,7 @@

View File

@ -0,0 +1 @@
SELECT substringUTF8(materialize(''), -9223372036854775808) FROM numbers(7);

View File

@ -0,0 +1 @@
SELECT toDate('2105-12-31') + INTERVAL number MONTH FROM system.numbers LIMIT 25000 FORMAT Null;

View File

@ -0,0 +1 @@
SELECT greatCircleAngle(0, -9223372036854775808, number, number) FROM numbers(3) FORMAT Null;

View File

@ -0,0 +1 @@
-0

View File

@ -0,0 +1 @@
SELECT round(avgWeighted(x, y)) FROM (SELECT 1023 AS x, 1000000000 AS y UNION ALL SELECT 10 AS x, -9223372036854775808 AS y);

View File

@ -0,0 +1,7 @@
Hello, world!
Hello, world!
1 \r\n\t\\\n
---
0 \\
---
0 \\

View File

@ -0,0 +1,40 @@
CREATE TEMPORARY TABLE test ("\\" String DEFAULT '\r\n\t\\' || '
');
INSERT INTO test VALUES ('Hello, world!');
INSERT INTO test ("\\") VALUES ('Hello, world!');
SELECT * FROM test;
DROP TEMPORARY TABLE test;
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt64, "\\" String DEFAULT '\r\n\t\\' || '
') ENGINE = MergeTree ORDER BY x;
INSERT INTO test (x) VALUES (1);
SELECT * FROM test;
DROP TABLE test;
DROP TABLE IF EXISTS test_r1;
DROP TABLE IF EXISTS test_r2;
CREATE TABLE test_r1 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || '
') ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r1') ORDER BY "\\";
INSERT INTO test_r1 ("\\") VALUES ('\\');
CREATE TABLE test_r2 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || '
') ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r2') ORDER BY "\\";
SYSTEM SYNC REPLICA test_r2;
SELECT '---';
SELECT * FROM test_r1;
SELECT '---';
SELECT * FROM test_r2;
DROP TABLE test_r1;
DROP TABLE test_r2;

View File

@ -191,3 +191,4 @@
01655_agg_if_nullable
01182_materialized_view_different_structure
01660_sum_ubsan
01669_columns_declaration_serde

View File

@ -1,3 +1 @@
# Suppress some failures in contrib so that we can enable UBSan in CI.
# Ideally, we should report these upstream.
src:*/contrib/openssl/*
# We have no suppressions!

View File

@ -126,3 +126,7 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
# Conflict markers
find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' |
xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files"
# Forbid subprocess.check_call(...) in integration tests because it does not provide enough information on errors
find $ROOT_PATH'/tests/integration' -name '*.py' |
xargs grep -F 'subprocess.check_call' | grep -v "STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL" && echo "Use helpers.cluster.run_and_check or subprocess.run instead of subprocess.check_call to print detailed info on error"