Fix terribly wrong code

This commit is contained in:
Alexey Milovidov 2020-09-19 02:04:36 +03:00
parent 37facf4482
commit ef5363b87f
13 changed files with 225 additions and 228 deletions

View File

@ -14,7 +14,7 @@
# pragma clang diagnostic ignored "-Wunused-macros"
#endif
#define __msan_unpoison(X, Y)
#define __msan_unpoison(X, Y) // NOLINT
#if defined(__has_feature)
# if __has_feature(memory_sanitizer)
# undef __msan_unpoison
@ -84,7 +84,7 @@ extern "C"
#ifdef ADDRESS_SANITIZER
void __lsan_ignore_object(const void *);
#else
void __lsan_ignore_object(const void *) {}
void __lsan_ignore_object(const void *) {} // NOLINT
#endif
}

View File

@ -43,7 +43,6 @@
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/SystemLog.cpp>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>

View File

@ -67,6 +67,7 @@ set(dbms_sources)
add_headers_and_sources(clickhouse_common_io Common)
add_headers_and_sources(clickhouse_common_io Common/HashTable)
add_headers_and_sources(clickhouse_common_io IO)
add_headers_and_sources(clickhouse_common_io IO/S3)
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
if(USE_RDKAFKA)

View File

@ -59,7 +59,7 @@ Otherwise you will get only exported symbols from program headers.
# pragma clang diagnostic ignored "-Wunused-macros"
#endif
#define __msan_unpoison_string(X)
#define __msan_unpoison_string(X) // NOLINT
#if defined(__has_feature)
# if __has_feature(memory_sanitizer)
# undef __msan_unpoison_string

View File

@ -187,11 +187,6 @@ static inline size_t tabulation(UInt64 x)
return res;
}
static inline size_t _intHash64(UInt64 x)
{
return static_cast<size_t>(intHash64(x));
}
const size_t BUF_SIZE = 1024;
@ -284,7 +279,7 @@ int main(int argc, char ** argv)
if (!method || method == 1) test<identity> (n, data.data(), "0: identity");
if (!method || method == 2) test<intHash32> (n, data.data(), "1: intHash32");
if (!method || method == 3) test<_intHash64>(n, data.data(), "2: intHash64");
if (!method || method == 3) test<intHash64> (n, data.data(), "2: intHash64");
if (!method || method == 4) test<hash3> (n, data.data(), "3: two rounds");
if (!method || method == 5) test<hash4> (n, data.data(), "4: two rounds and two variables");
if (!method || method == 6) test<hash5> (n, data.data(), "5: two rounds with less ops");

View File

@ -40,74 +40,76 @@ DataTypePtr convertMySQLDataType(MultiEnum<MySQLDataTypesSupport> type_support,
{
// we expect mysql_data_type to be either "basic_type" or "type_with_params(param1, param2, ...)"
auto data_type = std::string_view(mysql_data_type);
const auto param_start_pos = data_type.find("(");
const auto param_start_pos = data_type.find('(');
const auto type_name = data_type.substr(0, param_start_pos);
DataTypePtr res = [&]() -> DataTypePtr {
if (type_name == "tinyint")
{
if (is_unsigned)
return std::make_shared<DataTypeUInt8>();
else
return std::make_shared<DataTypeInt8>();
}
if (type_name == "smallint")
{
if (is_unsigned)
return std::make_shared<DataTypeUInt16>();
else
return std::make_shared<DataTypeInt16>();
}
if (type_name == "int" || type_name == "mediumint")
{
if (is_unsigned)
return std::make_shared<DataTypeUInt32>();
else
return std::make_shared<DataTypeInt32>();
}
if (type_name == "bigint")
{
if (is_unsigned)
return std::make_shared<DataTypeUInt64>();
else
return std::make_shared<DataTypeInt64>();
}
if (type_name == "float")
return std::make_shared<DataTypeFloat32>();
if (type_name == "double")
return std::make_shared<DataTypeFloat64>();
if (type_name == "date")
return std::make_shared<DataTypeDate>();
if (type_name == "binary")
return std::make_shared<DataTypeFixedString>(length);
if (type_name == "datetime" || type_name == "timestamp")
{
if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64))
return std::make_shared<DataTypeDateTime>();
DataTypePtr res;
if (type_name == "timestamp" && scale == 0)
{
return std::make_shared<DataTypeDateTime>();
}
else if (type_name == "datetime" || type_name == "timestamp")
{
return std::make_shared<DataTypeDateTime64>(scale);
}
}
if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal"))
if (type_name == "tinyint")
{
if (is_unsigned)
res = std::make_shared<DataTypeUInt8>();
else
res = std::make_shared<DataTypeInt8>();
}
else if (type_name == "smallint")
{
if (is_unsigned)
res = std::make_shared<DataTypeUInt16>();
else
res = std::make_shared<DataTypeInt16>();
}
else if (type_name == "int" || type_name == "mediumint")
{
if (is_unsigned)
res = std::make_shared<DataTypeUInt32>();
else
res = std::make_shared<DataTypeInt32>();
}
else if (type_name == "bigint")
{
if (is_unsigned)
res = std::make_shared<DataTypeUInt64>();
else
res = std::make_shared<DataTypeInt64>();
}
else if (type_name == "float")
res = std::make_shared<DataTypeFloat32>();
else if (type_name == "double")
res = std::make_shared<DataTypeFloat64>();
else if (type_name == "date")
res = std::make_shared<DataTypeDate>();
else if (type_name == "binary")
res = std::make_shared<DataTypeFixedString>(length);
else if (type_name == "datetime" || type_name == "timestamp")
{
if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64))
{
if (precision <= DecimalUtils::maxPrecision<Decimal32>())
return std::make_shared<DataTypeDecimal<Decimal32>>(precision, scale);
else if (precision <= DecimalUtils::maxPrecision<Decimal64>())
return std::make_shared<DataTypeDecimal<Decimal64>>(precision, scale);
else if (precision <= DecimalUtils::maxPrecision<Decimal128>())
return std::make_shared<DataTypeDecimal<Decimal128>>(precision, scale);
res = std::make_shared<DataTypeDateTime>();
}
else if (type_name == "timestamp" && scale == 0)
{
res = std::make_shared<DataTypeDateTime>();
}
else if (type_name == "datetime" || type_name == "timestamp")
{
res = std::make_shared<DataTypeDateTime64>(scale);
}
}
else if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal"))
{
if (precision <= DecimalUtils::maxPrecision<Decimal32>())
res = std::make_shared<DataTypeDecimal<Decimal32>>(precision, scale);
else if (precision <= DecimalUtils::maxPrecision<Decimal64>())
res = std::make_shared<DataTypeDecimal<Decimal64>>(precision, scale);
else if (precision <= DecimalUtils::maxPrecision<Decimal128>())
res = std::make_shared<DataTypeDecimal<Decimal128>>(precision, scale);
}
else
{
/// Also String is fallback for all unknown types.
return std::make_shared<DataTypeString>();
}();
res = std::make_shared<DataTypeString>();
}
if (is_nullable)
res = std::make_shared<DataTypeNullable>(res);

View File

@ -4,7 +4,6 @@
#include <Interpreters/Context.h>
#include "DiskS3.h"
#include "Disks/DiskCacheWrapper.h"
#include "Disks/DiskCacheWrapper.cpp"
#include "Disks/DiskFactory.h"
#include "ProxyConfiguration.h"
#include "ProxyListConfiguration.h"

View File

@ -92,7 +92,7 @@ private:
src_offset = src_offsets[i];
dst_offset += src_length;
if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front())
if (src_length > 1 && dst_data[dst_offset - 2] != UInt8(trailing_char_str.front()))
{
dst_data[dst_offset - 1] = trailing_char_str.front();
dst_data[dst_offset] = 0;

View File

@ -3,7 +3,6 @@
#include <utility>
#include <IO/HTTPCommon.h>
#include <IO/S3/PocoHTTPResponseStream.h>
#include <IO/S3/PocoHTTPResponseStream.cpp>
#include <Common/Stopwatch.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/HttpResponse.h>
@ -101,7 +100,7 @@ void PocoHTTPClient::MakeRequestInternal(
EnumSize,
};
auto selectMetric = [&request](S3MetricType type)
auto select_metric = [&request](S3MetricType type)
{
const ProfileEvents::Event events_map[][2] = {
{ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds},
@ -128,12 +127,12 @@ void PocoHTTPClient::MakeRequestInternal(
throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED);
};
ProfileEvents::increment(selectMetric(S3MetricType::Count));
ProfileEvents::increment(select_metric(S3MetricType::Count));
const int MAX_REDIRECT_ATTEMPTS = 10;
static constexpr int max_redirect_attempts = 10;
try
{
for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt)
for (int attempt = 0; attempt < max_redirect_attempts; ++attempt)
{
Poco::URI poco_uri(uri);
@ -202,7 +201,7 @@ void PocoHTTPClient::MakeRequestInternal(
auto & response_body_stream = session->receiveResponse(poco_response);
watch.stop();
ProfileEvents::increment(selectMetric(S3MetricType::Microseconds), watch.elapsedMicroseconds());
ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds());
int status_code = static_cast<int>(poco_response.getStatus());
LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason());
@ -214,7 +213,7 @@ void PocoHTTPClient::MakeRequestInternal(
uri = location;
LOG_DEBUG(log, "Redirecting request to new location: {}", location);
ProfileEvents::increment(selectMetric(S3MetricType::Redirects));
ProfileEvents::increment(select_metric(S3MetricType::Redirects));
continue;
}
@ -240,11 +239,11 @@ void PocoHTTPClient::MakeRequestInternal(
if (status_code == 429 || status_code == 503)
{ // API throttling
ProfileEvents::increment(selectMetric(S3MetricType::Throttling));
ProfileEvents::increment(select_metric(S3MetricType::Throttling));
}
else
{
ProfileEvents::increment(selectMetric(S3MetricType::Errors));
ProfileEvents::increment(select_metric(S3MetricType::Errors));
}
}
else
@ -261,7 +260,7 @@ void PocoHTTPClient::MakeRequestInternal(
response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION);
response->SetClientErrorMessage(getCurrentExceptionMessage(false));
ProfileEvents::increment(selectMetric(S3MetricType::Errors));
ProfileEvents::increment(select_metric(S3MetricType::Errors));
}
}
}

View File

@ -12,9 +12,7 @@
# include <aws/s3/S3Client.h>
# include <aws/core/http/HttpClientFactory.h>
# include <IO/S3/PocoHTTPClientFactory.h>
# include <IO/S3/PocoHTTPClientFactory.cpp>
# include <IO/S3/PocoHTTPClient.h>
# include <IO/S3/PocoHTTPClient.cpp>
# include <boost/algorithm/string.hpp>
# include <Poco/URI.h>
# include <re2/re2.h>

View File

@ -3,7 +3,7 @@
#include <Columns/ColumnVector.h>
// I know that inclusion of .cpp is not good at all
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp>
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp> // NOLINT
using namespace DB;
static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes)

View File

@ -3,7 +3,7 @@
#include <Columns/ColumnVector.h>
// I know that inclusion of .cpp is not good at all
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp>
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp> // NOLINT
using namespace DB;

View File

@ -6,7 +6,6 @@
#include <Core/Field.h>
#include <Core/Types.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.cpp>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
@ -16,6 +15,7 @@
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
using ColumnType = uint32_t;
using TableAndColumn = std::pair<std::string, std::string>;
pcg64 rng;
@ -92,7 +92,7 @@ TableAndColumn get_table_a_column(const std::string & c)
}
enum type : ColumnType
enum Type : ColumnType
{
i = 1,
// int
@ -114,9 +114,11 @@ enum type : ColumnType
};
std::map<ColumnType, std::string> type_definition = {
{type::i, "Int64"}, {type::f, "Float64"}, {type::s, "String"}, {type::d, "Date"}, {type::dt, "DateTime"}, {type::b, "UInt8"}
std::map<ColumnType, std::string> type_definition =
{
{Type::i, "Int64"}, {Type::f, "Float64"}, {Type::s, "String"}, {Type::d, "Date"}, {Type::dt, "DateTime"}, {Type::b, "UInt8"}
};
ColumnType time_type(std::string value)
{
if (value.length() == 12)
@ -124,14 +126,14 @@ ColumnType time_type(std::string value)
for (size_t i : {5, 8})
{
if (value[i] != '-')
return type::s;
return Type::s;
}
for (size_t i : {1, 2, 3, 4, 6, 7, 9, 10})
{
if (!isdigit(value[i]))
return type::s;
return Type::s;
}
return type::d;
return Type::d;
}
if (value.length() == 21)
@ -139,18 +141,18 @@ ColumnType time_type(std::string value)
for (size_t i : {5, 8})
{
if (value[i] != '-')
return type::s;
return Type::s;
}
for (size_t i : {14, 17})
{
if (value[i] != '-')
return type::s;
return Type::s;
}
if (value[11] != '-')
return type::s;
return type::dt;
return Type::s;
return Type::dt;
}
return type::s;
return Type::s;
}
// Casting inner clickhouse parser type to our type
ColumnType type_cast(int t)
@ -164,21 +166,21 @@ ColumnType type_cast(int t)
case 19:
case 20:
case 21:
return type::i;
return Type::i;
case 3:
return type::f;
return Type::f;
case 16:
return type::s;
return Type::s;
case 17:
return type::a | type::all;
return Type::a | Type::all;
case 18:
return type::t | type::all;
return Type::t | Type::all;
}
return type::all;
return Type::all;
}
@ -197,28 +199,28 @@ public:
, is_array(is_a) {}
std::string value{};
ColumnType type = type::all;
ColumnType type = Type::all;
bool is_array = false;
};
std::map<std::string, FuncRet> func_to_return_type = {
{"divide", FuncRet(type::f, "")}, {"e", FuncRet(type::f, "e()")}, {"pi", FuncRet(type::f, "pi()")}, {"exp", FuncRet(type::f, "")},
{"log", FuncRet(type::f,"")}, {"exp2", FuncRet(type::f, "")}, {"log2", FuncRet(type::f, "")}, {"exp10", FuncRet(type::f, "")},
{"log10", FuncRet(type::f, "")}, {"sqrt", FuncRet(type::f, "")}, {"cbrt", FuncRet(type::f, "")}, {"erf", FuncRet(type::f, "")},
{"erfc", FuncRet(type::f, "")}, {"lgamma", FuncRet(type::f, "")}, {"tgamma", FuncRet(type::f, "")}, {"sin", FuncRet(type::f, "")},
{"cos", FuncRet(type::f, "")}, {"tan", FuncRet(type::f, "")}, {"asin", FuncRet(type::f, "")}, {"acos", FuncRet(type::f, "")},
{"atan", FuncRet(type::f, "")}, {"pow", FuncRet(type::f, "")}, {"splitbystring", FuncRet(type::s | type::a,"")},
{"splitbychar", FuncRet(type::s | type::a, "")}, {"alphatokens", FuncRet(type::s | type::a, "")}, {"toyear", FuncRet(type::i, "")},
{"tomonth", FuncRet(type::i, "")}, {"todayofmonth", FuncRet(type::i, "")}, {"tohour", FuncRet(type::dt, "")}, {"tominute", FuncRet(type::dt, "")},
{"toseconds", FuncRet(type::dt, "")}, {"tounixtimestamp", FuncRet(type::i, "")}, {"tostartofyear", FuncRet(type::dt | type::d, "")},
{"tostartofquater",FuncRet(type::dt | type::d, "")}, {"tostartofmonth", FuncRet(type::dt | type::d, "")}, {"tomonday", FuncRet(type::dt | type::d, "")},
{"tostartoffiveminutes", FuncRet(type::dt, "")}, {"tostartoftenminutes", FuncRet(type::dt, "")}, {"tostartoffifteenminutes", FuncRet(type::dt, "")},
{"tostartofinterval", FuncRet(type::dt, "")}, {"totime", FuncRet(type::dt, "")}, {"torelativemonthnum", FuncRet(type::i, "")},
{"torelativeweeknum", FuncRet(type::i, "")}, {"torelativedaynum", FuncRet(type::i, "")}, {"torelativehournum", FuncRet(type::i, "")},
{"torelativeminutenum", FuncRet(type::i, "")}, {"torelativesecondsnum", FuncRet(type::i, "")}, {"datediff", FuncRet(type::d | type::dt, "")},
{"formatdatetime", FuncRet(type::s, "")}, {"now", FuncRet(type::dt | type::d, "now()")}, {"today", FuncRet(type::d | type::dt, "today()")},
{"yesterday", FuncRet(type::d | type::dt, "yesterday()")}
{"divide", FuncRet(Type::f, "")}, {"e", FuncRet(Type::f, "e()")}, {"pi", FuncRet(Type::f, "pi()")}, {"exp", FuncRet(Type::f, "")},
{"log", FuncRet(Type::f,"")}, {"exp2", FuncRet(Type::f, "")}, {"log2", FuncRet(Type::f, "")}, {"exp10", FuncRet(Type::f, "")},
{"log10", FuncRet(Type::f, "")}, {"sqrt", FuncRet(Type::f, "")}, {"cbrt", FuncRet(Type::f, "")}, {"erf", FuncRet(Type::f, "")},
{"erfc", FuncRet(Type::f, "")}, {"lgamma", FuncRet(Type::f, "")}, {"tgamma", FuncRet(Type::f, "")}, {"sin", FuncRet(Type::f, "")},
{"cos", FuncRet(Type::f, "")}, {"tan", FuncRet(Type::f, "")}, {"asin", FuncRet(Type::f, "")}, {"acos", FuncRet(Type::f, "")},
{"atan", FuncRet(Type::f, "")}, {"pow", FuncRet(Type::f, "")}, {"splitbystring", FuncRet(Type::s | Type::a,"")},
{"splitbychar", FuncRet(Type::s | Type::a, "")}, {"alphatokens", FuncRet(Type::s | Type::a, "")}, {"toyear", FuncRet(Type::i, "")},
{"tomonth", FuncRet(Type::i, "")}, {"todayofmonth", FuncRet(Type::i, "")}, {"tohour", FuncRet(Type::dt, "")}, {"tominute", FuncRet(Type::dt, "")},
{"toseconds", FuncRet(Type::dt, "")}, {"tounixtimestamp", FuncRet(Type::i, "")}, {"tostartofyear", FuncRet(Type::dt | Type::d, "")},
{"tostartofquater",FuncRet(Type::dt | Type::d, "")}, {"tostartofmonth", FuncRet(Type::dt | Type::d, "")}, {"tomonday", FuncRet(Type::dt | Type::d, "")},
{"tostartoffiveminutes", FuncRet(Type::dt, "")}, {"tostartoftenminutes", FuncRet(Type::dt, "")}, {"tostartoffifteenminutes", FuncRet(Type::dt, "")},
{"tostartofinterval", FuncRet(Type::dt, "")}, {"totime", FuncRet(Type::dt, "")}, {"torelativemonthnum", FuncRet(Type::i, "")},
{"torelativeweeknum", FuncRet(Type::i, "")}, {"torelativedaynum", FuncRet(Type::i, "")}, {"torelativehournum", FuncRet(Type::i, "")},
{"torelativeminutenum", FuncRet(Type::i, "")}, {"torelativesecondsnum", FuncRet(Type::i, "")}, {"datediff", FuncRet(Type::d | Type::dt, "")},
{"formatdatetime", FuncRet(Type::s, "")}, {"now", FuncRet(Type::dt | Type::d, "now()")}, {"today", FuncRet(Type::d | Type::dt, "today()")},
{"yesterday", FuncRet(Type::d | Type::dt, "yesterday()")}
};
std::set<std::string> func_args_same_types = {
@ -226,23 +228,23 @@ std::set<std::string> func_args_same_types = {
};
std::map<std::string, ColumnType> func_to_param_type = {
{"tostartofminute", type::dt}, {"plus", type::i | type::f | type::d | type::dt}, {"multiply", type::i | type::f},
{"minus", type::i | type::f | type::d | type::dt}, {"negate", type::i | type::f}, {"divide", type::i | type::f},
{"abs", type::i | type::f}, {"gcd", type::i | type::f}, {"lcm", type::i | type::f}, {"bitnot", type::i}, {"bitshiftleft", type::i},
{"bitshiftright", type::i}, {"bittest", type::i}, {"exp", type::i | type::f}, {"log", type::i | type::f},
{"exp2", type::i | type::f}, {"log2", type::i | type::f}, {"exp10", type::i | type::f}, {"log10", type::i | type::f},
{"sqrt", type::i | type::f}, {"cbrt", type::i | type::f}, {"erf", type::i | type::f}, {"erfc", type::i | type::f},
{"lgamma", type::i | type::f}, {"tgamma", type::i | type::f}, {"sin", type::i | type::f}, {"cos", type::i | type::f},
{"tan", type::i | type::f}, {"asin", type::i | type::f}, {"acos", type::i | type::f}, {"atan", type::i | type::f},
{"pow", type::i | type::f}, {"arrayjoin", type::all | type::a}, {"substring", type::s}, {"splitbystring", type::s}, {"splitbychar", type::s},
{"alphatokens", type::s}, {"toyear", type::d | type::dt}, {"tomonth", type::d | type::dt}, {"todayofmonth", type::d | type::dt}, {"tohour", type::dt},
{"tominute", type::dt}, {"tosecond", type::dt}, {"touixtimestamp", type::dt}, {"tostartofyear", type::d | type::dt},
{"tostartofquarter", type::d | type::dt}, {"tostartofmonth", type::d | type::dt}, {"tomonday", type::d | type::dt},
{"tostartoffiveminute", type::dt}, {"tostartoftenminutes", type::dt}, {"tostartoffifteenminutes", type::d | type::dt},
{"tostartofinterval", type::d | type::dt}, {"totime", type::d | type::dt}, {"torelativehonthnum", type::d | type::dt},
{"torelativeweeknum", type::d | type::dt}, {"torelativedaynum", type::d | type::dt}, {"torelativehournum", type::d | type::dt},
{"torelativeminutenum", type::d | type::dt}, {"torelativesecondnum", type::d | type::dt}, {"datediff", type::d | type::dt},
{"formatdatetime", type::dt}
{"tostartofminute", Type::dt}, {"plus", Type::i | Type::f | Type::d | Type::dt}, {"multiply", Type::i | Type::f},
{"minus", Type::i | Type::f | Type::d | Type::dt}, {"negate", Type::i | Type::f}, {"divide", Type::i | Type::f},
{"abs", Type::i | Type::f}, {"gcd", Type::i | Type::f}, {"lcm", Type::i | Type::f}, {"bitnot", Type::i}, {"bitshiftleft", Type::i},
{"bitshiftright", Type::i}, {"bittest", Type::i}, {"exp", Type::i | Type::f}, {"log", Type::i | Type::f},
{"exp2", Type::i | Type::f}, {"log2", Type::i | Type::f}, {"exp10", Type::i | Type::f}, {"log10", Type::i | Type::f},
{"sqrt", Type::i | Type::f}, {"cbrt", Type::i | Type::f}, {"erf", Type::i | Type::f}, {"erfc", Type::i | Type::f},
{"lgamma", Type::i | Type::f}, {"tgamma", Type::i | Type::f}, {"sin", Type::i | Type::f}, {"cos", Type::i | Type::f},
{"tan", Type::i | Type::f}, {"asin", Type::i | Type::f}, {"acos", Type::i | Type::f}, {"atan", Type::i | Type::f},
{"pow", Type::i | Type::f}, {"arrayjoin", Type::all | Type::a}, {"substring", Type::s}, {"splitbystring", Type::s}, {"splitbychar", Type::s},
{"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt},
{"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt},
{"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt},
{"tostartoffiveminute", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt},
{"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt},
{"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt},
{"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt},
{"formatdatetime", Type::dt}
};
@ -252,7 +254,7 @@ public:
TableAndColumn name;
std::set<TableAndColumn> equals;
std::set<std::string> values;
ColumnType type = type::all;
ColumnType type = Type::all;
bool is_array = false;
Column() = default;
@ -260,7 +262,7 @@ public:
explicit Column(const std::string & column_name)
{
name = std::make_pair("", column_name);
type = type::all;
type = Type::all;
}
void merge(Column other)
@ -275,15 +277,15 @@ public:
void printType() const
{
if (type & type::i)
if (type & Type::i)
std::cout << "I";
if (type & type::f)
if (type & Type::f)
std::cout << "F";
if (type & type::s)
if (type & Type::s)
std::cout << "S";
if (type & type::d)
if (type & Type::d)
std::cout << "D";
if (type & type::dt)
if (type & Type::dt)
std::cout << "DT";
if (is_array)
std::cout << "ARR";
@ -307,22 +309,22 @@ public:
std::string generateOneValue() const
{
if (type & type::i)
if (type & Type::i)
return randomInteger();
if (type & type::f)
if (type & Type::f)
return randomFloat();
if (type & type::d)
if (type & Type::d)
return randomDate();
if (type & type::dt)
if (type & Type::dt)
return randomDatetime();
if (type & type::s)
if (type & Type::s)
return "'" + randomString(rng() % 40) + "'";
if (type & type::b)
if (type & Type::b)
return "0";
return "";
@ -332,7 +334,7 @@ public:
{
if (values.size() > 2 && amount == 0)
return false;
while (values.size() < 1 or amount > 0)
while (values.empty() or amount > 0)
{
amount -= 1;
if (is_array)
@ -357,18 +359,18 @@ public:
void unifyType()
{
if (type & type::i)
type = type::i;
else if (type & type::f)
type = type::f;
else if (type & type::d)
type = type::d;
else if (type & type::dt)
type = type::dt;
else if (type & type::s)
type = type::s;
else if (type & type::b)
type = type::b;
if (type & Type::i)
type = Type::i;
else if (type & Type::f)
type = Type::f;
else if (type & Type::d)
type = Type::d;
else if (type & Type::dt)
type = Type::dt;
else if (type & Type::s)
type = Type::s;
else if (type & Type::b)
type = Type::b;
else
throw std::runtime_error("Error in determination column type " + name.first + '.' + name.second);
}
@ -381,13 +383,15 @@ decartMul(
std::set<std::string> & mul)
{
std::set<std::vector<std::string>> result;
for (auto v : prev)
for (auto m : mul)
for (const auto & v : prev)
{
for (const auto & m : mul)
{
std::vector<std::string> tmp = v;
tmp.push_back(m);
result.insert(tmp);
}
}
return result;
}
@ -438,7 +442,7 @@ public:
{
name = other.name;
columns.insert(other.columns.begin(), other.columns.end());
for (auto desc : other.column_description)
for (const auto & desc : other.column_description)
column_description[desc.first].merge(desc.second);
}
@ -576,7 +580,7 @@ public:
void merge(TableList other)
{
for (auto table : other.tables)
for (const auto & table : other.tables)
tables[table.first].merge(table.second);
nested.insert(other.nested.begin(), other.nested.end());
if (main_table.empty())
@ -617,14 +621,14 @@ FuncRet arrayJoinFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
for (const auto & indent : indents)
{
auto c = Column(indent);
c.type = type::all;
c.type = Type::all;
c.is_array = true;
if (columns.count(indent))
columns[indent].merge(c);
else
columns[indent] = c;
}
FuncRet r(type::all, "");
FuncRet r(Type::all, "");
return r;
}
return FuncRet();
@ -637,7 +641,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
std::set<std::string> indents{};
std::set<std::string> values{};
ColumnType type_value = type::all;
ColumnType type_value = Type::all;
for (auto & arg : x->arguments->children)
{
@ -656,20 +660,20 @@ FuncRet inFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
for (auto & val : arr_values)
{
type = type_cast(val.getType());
if (type == type::s || type == type::d || type == type::dt)
if (type == Type::s || type == Type::d || type == Type::dt)
type = time_type(applyVisitor(DB::FieldVisitorToString(), val));
type_value &= type;
values.insert(applyVisitor(DB::FieldVisitorToString(), val));
}
};
if (type & type::a)
if (type & Type::a)
{
auto arr_values = literal->value.get<DB::Array>();
routine(arr_values);
}
if (type & type::a)
if (type & Type::a)
{
auto arr_values = literal->value.get<DB::Tuple>();
routine(arr_values);
@ -685,7 +689,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
else
f = handlers[""];
FuncRet ret = f(arg, columns);
if (ret.value != "")
if (!ret.value.empty())
{
values.insert(ret.value);
}
@ -703,7 +707,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
else
columns[indent] = c;
}
FuncRet r(type::b | type::i, "");
FuncRet r(Type::b | Type::i, "");
return r;
}
return FuncRet();
@ -716,7 +720,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
std::set<std::string> indents = {};
std::string value = "[";
ColumnType type_value = type::i | type::f | type::d | type::dt | type::s;
ColumnType type_value = Type::i | Type::f | Type::d | Type::dt | Type::s;
bool no_indent = true;
for (const auto & arg : x->arguments->children)
{
@ -730,7 +734,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
if (literal)
{
ColumnType type = type_cast(literal->value.getType());
if (type == type::s || type == type::d || type == type::dt)
if (type == Type::s || type == Type::d || type == Type::dt)
type = time_type(value);
type_value &= type;
@ -764,7 +768,7 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
std::set<std::string> indents = {};
std::set<std::string> values = {};
ColumnType type_value = type::i | type::f | type::d | type::dt;
ColumnType type_value = Type::i | Type::f | Type::d | Type::dt;
ColumnType args_types = 0;
bool no_indent = true;
for (auto & arg : x->arguments->children)
@ -793,11 +797,11 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
}
args_types |= type;
}
if (args_types & (type::d | type::dt))
type_value -= type::f;
if (args_types & type::f)
type_value -= type::d | type::dt;
for (auto indent : indents)
if (args_types & (Type::d | Type::dt))
type_value -= Type::f;
if (args_types & Type::f)
type_value -= Type::d | Type::dt;
for (const auto & indent : indents)
{
auto c = Column(indent);
c.type = type_value;
@ -807,14 +811,14 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
columns[indent] = c;
}
ColumnType ret_type = 0;
if (args_types & type::dt)
ret_type = type::dt;
else if (args_types & type::d)
ret_type = type::d | type::dt;
else if (args_types & type::f)
ret_type = type::f;
if (args_types & Type::dt)
ret_type = Type::dt;
else if (args_types & Type::d)
ret_type = Type::d | Type::dt;
else if (args_types & Type::f)
ret_type = Type::f;
else
ret_type = type::d | type::f | type::dt | type::i;
ret_type = Type::d | Type::f | Type::dt | Type::i;
FuncRet r(ret_type, "");
if (no_indent)
{
@ -833,7 +837,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
std::set<std::string> indents = {};
std::set<std::string> values = {};
ColumnType type_value = type::s;
ColumnType type_value = Type::s;
for (auto & arg : x->arguments->children)
{
auto ident = std::dynamic_pointer_cast<DB::ASTIdentifier>(arg);
@ -866,7 +870,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
else
columns[indent] = c;
}
FuncRet r(type::b, "");
FuncRet r(Type::b, "");
return r;
}
return FuncRet();
@ -874,22 +878,22 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
auto X = std::dynamic_pointer_cast<DB::ASTFunction>(ch);
if (X)
auto x = std::dynamic_pointer_cast<DB::ASTFunction>(ch);
if (x)
{
std::set<std::string> indents = {};
std::set<std::string> values = {};
ColumnType type_value = type::all;
ColumnType type_value = Type::all;
bool is_array = false;
bool no_indent = true;
if (func_to_param_type.count(boost::algorithm::to_lower_copy(X->name)))
if (func_to_param_type.count(boost::algorithm::to_lower_copy(x->name)))
{
type_value &= func_to_param_type[boost::algorithm::to_lower_copy(X->name)];
is_array = func_to_param_type[boost::algorithm::to_lower_copy(X->name)] & type::a;
type_value &= func_to_param_type[boost::algorithm::to_lower_copy(x->name)];
is_array = func_to_param_type[boost::algorithm::to_lower_copy(x->name)] & Type::a;
}
for (auto arg : X->arguments->children)
for (const auto & arg : x->arguments->children)
{
ColumnType type = type::all;
ColumnType type = Type::all;
std::string value;
auto ident = std::dynamic_pointer_cast<DB::ASTIdentifier>(arg);
if (ident)
@ -902,7 +906,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
{
value = applyVisitor(DB::FieldVisitorToString(), literal->value);
type = type_cast(literal->value.getType());
is_array |= type & type::a;
is_array |= type & Type::a;
}
auto subfunc = std::dynamic_pointer_cast<DB::ASTFunction>(arg);
if (subfunc)
@ -922,31 +926,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
}
if (!value.empty())
{
if (type == type::i)
if (type == Type::i)
{
values.insert(value);
values.insert(value + " + " + randomInteger(1, 10));
values.insert(value + " - " + randomInteger(1, 10));
}
if (type == type::f)
if (type == Type::f)
{
values.insert(value);
values.insert(value + " + " + randomFloat(1, 10));
values.insert(value + " - " + randomFloat(1, 10));
}
if (type & type::s || type & type::d || type & type::dt)
if (type & Type::s || type & Type::d || type & Type::dt)
{
if (type == type::s)
if (type == Type::s)
type = time_type(value);
if (type == type::s)
if (type == Type::s)
values.insert(value);
if (type & type::d)
if (type & Type::d)
{
values.insert(value);
values.insert("toDate(" + value + ") + " + randomInteger(1, 10));
values.insert("toDate(" + value + ") - " + randomInteger(1, 10));
}
else if (type & type::dt)
else if (type & Type::dt)
{
values.insert(value);
values.insert(
@ -956,7 +960,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
}
}
}
if (func_args_same_types.count(boost::algorithm::to_lower_copy(X->name)))
if (func_args_same_types.count(boost::algorithm::to_lower_copy(x->name)))
type_value &= type;
}
for (const auto & indent : indents)
@ -965,7 +969,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
c.type = type_value;
c.is_array = is_array;
if (func_args_same_types.count(
boost::algorithm::to_lower_copy(X->name)))
boost::algorithm::to_lower_copy(x->name)))
c.values = values;
for (const auto & ind : indents)
if (ind != indent)
@ -976,31 +980,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map<std::string, Column> & columns)
else
columns[indent] = c;
}
if (func_to_return_type.count(boost::algorithm::to_lower_copy(X->name)))
if (func_to_return_type.count(boost::algorithm::to_lower_copy(x->name)))
{
if (no_indent)
{
std::ostringstream ss;
formatAST(*ch, ss);
auto r = func_to_return_type[boost::algorithm::to_lower_copy(X->name)];
auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)];
r.value = ss.str();
return r;
}
return func_to_return_type[boost::algorithm::to_lower_copy(X->name)];
return func_to_return_type[boost::algorithm::to_lower_copy(x->name)];
}
else if (func_to_param_type.count(
boost::algorithm::to_lower_copy(X->name)))
boost::algorithm::to_lower_copy(x->name)))
{
if (no_indent)
{
std::ostringstream ss;
formatAST(*ch, ss);
return FuncRet(
func_to_param_type[boost::algorithm::to_lower_copy(X->name)],
func_to_param_type[boost::algorithm::to_lower_copy(x->name)],
ss.str());
}
return FuncRet(
func_to_param_type[boost::algorithm::to_lower_copy(X->name)],
func_to_param_type[boost::algorithm::to_lower_copy(x->name)],
"");
}
}
@ -1070,7 +1074,7 @@ connectedEqualityFind(
std::set<TableAndColumn> & visited)
{
std::set<TableAndColumn> result;
for (auto & column : now.equals)
for (const auto & column : now.equals)
if (!visited.count(column))
{
visited.insert(column);
@ -1117,18 +1121,18 @@ unificateColumns(
result[column.second.name.first + "." + column.second.name.second],
result,
visited);
for (auto c : equal)
for (const auto & c : equal)
result[c.first + "." + c.second].equals = equal;
}
for (auto & column : result)
for (auto e : column.second.equals)
for (const auto & e : column.second.equals)
column.second.merge(result[e.first + "." + e.second]);
for (auto & column : result)
{
column.second.unifyType();
if (column.second.generateValues())
for (auto e : column.second.equals)
for (const auto & e : column.second.equals)
result[e.first + "." + e.second].merge(column.second);
}
@ -1174,18 +1178,18 @@ void parseSelectQuery(DB::ASTPtr ast, TableList & all_tables)
for (auto & child : x->children)
{
auto ch = std::dynamic_pointer_cast<DB::ASTTablesInSelectQueryElement>(child);
auto TEast = std::dynamic_pointer_cast<DB::ASTTableExpression>(ch->table_expression);
if (TEast && TEast->database_and_table_name)
auto table_expression_ast = std::dynamic_pointer_cast<DB::ASTTableExpression>(ch->table_expression);
if (table_expression_ast && table_expression_ast->database_and_table_name)
{
auto table_name = *(getIndent(TEast->database_and_table_name).begin());
auto table_name = *(getIndent(table_expression_ast->database_and_table_name).begin());
all_tables.addTable(table_name);
auto alias = getAlias(ch);
if (!alias.empty())
all_tables.aliases[alias] = table_name;
}
if (TEast && TEast->subquery)
if (table_expression_ast && table_expression_ast->subquery)
{
for (auto select : getSelect(TEast->subquery))
for (const auto & select : getSelect(table_expression_ast->subquery))
{
TableList local;
parseSelectQuery(select, local);