mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix terribly wrong code
This commit is contained in:
parent
37facf4482
commit
ef5363b87f
@ -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
|
||||
}
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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");
|
||||
|
@ -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);
|
||||
|
@ -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"
|
||||
|
@ -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;
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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>
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user