提交 c062fc1c 编写于 作者: A Alexey Milovidov

dbms: cutting dependency [#METR-2944].

上级 13c7d098
......@@ -17,6 +17,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
/// Частный случай - реализация для числовых типов.
template <typename T>
......
......@@ -15,6 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_SLOW;
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_MUCH_ARGUMENTS_FOR_FUNCTION;
extern const int SYNTAX_ERROR;
}
/// helper type for comparing `std::pair`s using solely the .first member
template <template <typename> class Comparator>
struct ComparePairFirst final
......
......@@ -16,6 +16,7 @@ namespace ErrorCodes
{
extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
using AggregateDataPtr = char *;
......
......@@ -20,6 +20,16 @@
/// Вызов quantile занимает O(sample_count log sample_count), если после предыдущего вызова quantile был хотя бы один вызов insert. Иначе, O(1).
/// То есть, имеет смысл сначала добавлять, потом получать квантили, не добавляя.
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
}
}
namespace detail
{
const size_t DEFAULT_SAMPLE_COUNT = 8192;
......
......@@ -12,6 +12,12 @@ namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_PARAMETER;
}
/// Performs case-sensitive and case-insensitive search of UTF-8 strings
template <bool CaseSensitive, bool ASCII> class StringSearcher;
......
......@@ -23,6 +23,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
class CacheDictionary final : public IDictionary
{
public:
......
......@@ -14,6 +14,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
const auto max_connections = 16;
/** Allows loading dictionaries from local or remote ClickHouse instance
......
......@@ -26,6 +26,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
class ComplexKeyCacheDictionary final : public IDictionaryBase
{
public:
......
......@@ -16,6 +16,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
class ComplexKeyHashedDictionary final : public IDictionaryBase
{
public:
......
......@@ -15,6 +15,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int LOGICAL_ERROR;
}
namespace
{
......
......@@ -19,6 +19,7 @@ namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TYPE_MISMATCH;
}
......
......@@ -15,6 +15,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
const auto initial_array_size = 1024;
const auto max_array_size = 500000;
......
......@@ -14,6 +14,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
class HashedDictionary final : public IDictionary
{
public:
......
......@@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int WRONG_PASSWORD;
extern const int MONGODB_INIT_FAILED;
}
/// Allows loading dictionaries from a MongoDB collection
class MongoDBDictionarySource final : public IDictionarySource
{
......
......@@ -16,6 +16,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IProfilingBlockInputStream
{
......
......@@ -11,6 +11,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
/// Allows loading dictionaries from a MySQL database
class MySQLDictionarySource final : public IDictionarySource
{
......
......@@ -11,6 +11,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_DIVISION;
}
/** Арифметические функции: +, -, *, /, %,
* intDiv (целочисленное деление), унарный минус.
* Битовые функции: |, &, ^, ~.
......
......@@ -28,6 +28,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_NUMBER;
}
/** Функции преобразования типов.
* toType - преобразование "естественным образом";
*/
......
......@@ -29,6 +29,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int DICTIONARIES_WAS_NOT_LOADED;
}
/** Функции, использующие словари Яндекс.Метрики
* - словари регионов, операционных систем, поисковых систем.
*
......
......@@ -14,6 +14,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
}
/** Функции высшего порядка для массивов:
*
* arrayMap(x1,...,xn -> expression, array1,...,arrayn) - применить выражение к каждому элементу массива (или набора параллельных массивов).
......
......@@ -10,6 +10,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_CLOCK_GETTIME;
}
/** Функции генерации псевдослучайных чисел.
* Функция может быть вызвана без аргументов или с одним аргументом.
* Аргумент игнорируется и служит лишь для того, чтобы несколько вызовов одной функции считались разными и не склеивались.
......
......@@ -17,6 +17,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
}
class ExpressionAction;
......
......@@ -21,6 +21,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
}
/** Позволяет писать файл на удалённый сервер.
*/
class RemoteWriteBuffer : public WriteBuffer
......@@ -63,7 +69,7 @@ public:
{
Poco::URI::encode(path, "&#", encoded_path);
Poco::URI::encode(tmp_path, "&#", encoded_tmp_path);
std::stringstream uri;
uri << "http://" << host << ":" << port
<< "/?action=write"
......@@ -76,14 +82,14 @@ public:
session.setHost(host);
session.setPort(port);
session.setKeepAlive(true);
/// устанавливаем таймаут
session.setTimeout(connection_timeout, send_timeout, receive_timeout);
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri_str, Poco::Net::HTTPRequest::HTTP_1_1);
request.setChunkedTransferEncoding(true);
for (unsigned i = 0; i < connection_retries; ++i)
{
LOG_TRACE((&Logger::get("RemoteWriteBuffer")), "Sending write request to " << host << ":" << port << uri_str);
......@@ -126,7 +132,7 @@ public:
/// Для корректной работы с AsynchronousWriteBuffer, который подменяет буферы.
impl->set(buffer().begin(), buffer().size());
impl->position() = pos;
try
......@@ -145,7 +151,7 @@ public:
{
if (finalized)
return;
next();
checkStatus();
......@@ -193,7 +199,7 @@ private:
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
}
}
void rename()
{
std::stringstream uri;
......@@ -203,9 +209,9 @@ private:
<< "&to=" << encoded_path;
uri_str = Poco::URI(uri.str()).getPathAndQuery();
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri_str, Poco::Net::HTTPRequest::HTTP_1_1);
for (unsigned i = 0; i < connection_retries; ++i)
{
LOG_TRACE((&Logger::get("RemoteWriteBuffer")), "Sending rename request to " << host << ":" << port << uri_str);
......@@ -219,7 +225,7 @@ private:
{
if (i + 1 == connection_retries)
throw;
LOG_WARNING((&Logger::get("RemoteWriteBuffer")), e.what() << ", message: " << e.displayText()
<< ", URL: " << host << ":" << port << uri_str << ", try No " << i + 1 << ".");
session.reset();
......@@ -229,7 +235,7 @@ private:
{
if (i + 1 == connection_retries)
throw;
LOG_WARNING((&Logger::get("RemoteWriteBuffer")), "Connection timeout from " << host << ":" << port << uri_str << ", try No " << i + 1 << ".");
session.reset();
continue;
......
......@@ -7,6 +7,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_AGGREGATE_FUNCTION;
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
......
......@@ -78,8 +78,24 @@
namespace DB
{
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int NETWORK_ERROR;
extern const int NO_DATA_TO_INSERT;
extern const int BAD_ARGUMENTS;
extern const int CANNOT_READ_HISTORY;
extern const int CANNOT_APPEND_HISTORY;
extern const int UNKNOWN_PACKET_FROM_SERVER;
extern const int UNEXPECTED_PACKET_FROM_SERVER;
extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED;
}
using Poco::SharedPtr;
class Client : public Poco::Util::Application
{
public:
......
......@@ -3,6 +3,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int MISMATCH_REPLICAS_DATA_SOURCES;
extern const int NO_AVAILABLE_REPLICA;
extern const int TIMEOUT_EXCEEDED;
}
MultiplexedConnections::MultiplexedConnections(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_)
: settings(settings_), throttler(throttler_), supports_parallel_execution(false)
{
......
此差异已折叠。
......@@ -4,6 +4,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FUNCTION;
}
/** Эти функции определены в отдельных translation unit-ах.
* Это сделано для того, чтобы уменьшить потребление оперативки при сборке, и ускорить параллельную сборку.
*/
......
......@@ -8,6 +8,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int LOGICAL_ERROR;
extern const int SHARD_HAS_NO_CONNECTIONS;
}
namespace
{
......
......@@ -13,6 +13,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int UNSUPPORTED_METHOD;
extern const int TOO_SMALL_BUFFER_SIZE;
}
DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, Context & context) const
......
......@@ -6,13 +6,19 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Core/ColumnNumbers.h>
/*#include <DB/DataStreams/TabSeparatedBlockOutputStream.h>
*#include <DB/IO/WriteBufferFromFileDescriptor.h>*/
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SET_DATA_VARIANT;
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH;
extern const int ILLEGAL_COLUMN;
}
Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
{
......@@ -67,7 +73,7 @@ static void initImpl(Maps & maps, Join::Type type)
case Join::Type::CROSS: break;
default:
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
}
......
......@@ -28,6 +28,18 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SET_DATA_VARIANT;
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int INCORRECT_ELEMENT_OF_SET;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
void SetVariants::init(Type type_)
{
type = type_;
......@@ -42,7 +54,7 @@ void SetVariants::init(Type type_)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
}
......@@ -59,7 +71,7 @@ size_t SetVariants::getTotalRowCount() const
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
}
......@@ -76,7 +88,7 @@ size_t SetVariants::getTotalByteCount() const
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
}
......
......@@ -4,6 +4,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
extern const int THERE_IS_NO_PROFILE;
extern const int NO_ELEMENTS_IN_CONFIG;
}
/// Установить настройку по имени.
void Settings::set(const String & name, const Field & value)
{
......
......@@ -7,6 +7,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ABORTED;
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
}
void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
......
......@@ -19,6 +19,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
}
void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
HTMLForm params(request);
......
......@@ -5,6 +5,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_PARAMETER;
extern const int UNKNOWN_IDENTIFIER;
extern const int UNKNOWN_RELATION;
}
namespace OLAP
{
......@@ -12,7 +21,7 @@ QueryConverter::QueryConverter(Poco::Util::AbstractConfiguration & config)
{
table_for_single_counter = config.getString("olap_compatibility.table_for_single_counter");
table_for_all_counters = config.getString("olap_compatibility.table_for_all_counters");
fillFormattedAttributeMap();
fillNumericAttributeMap();
fillFormattingAggregatedAttributeMap();
......@@ -47,20 +56,20 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query,
/// Проверим, умеем ли мы выполнять такой запрос.
if (query.format != FORMAT_TAB)
throw Exception("Only tab-separated output format is supported", ErrorCodes::UNSUPPORTED_PARAMETER);
/// Учтем некоторые настройки (далеко не все).
Settings new_settings = inout_context.getSettings();
if (query.concurrency != 0)
new_settings.max_threads = query.concurrency;
if (query.max_execution_time != 0)
new_settings.limits.max_execution_time = Poco::Timespan(query.max_execution_time, 0);
if (query.max_result_size != 0)
new_settings.limits.max_rows_to_group_by = query.max_result_size;
if (query.has_overflow_mode)
{
switch (query.overflow_mode)
......@@ -76,54 +85,54 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query,
break;
}
}
inout_context.setSettings(new_settings);
/// Составим запрос.
out_query = "SELECT ";
std::vector<std::string> selected_expressions;
/// Что выбирать: ключи агрегации и агрегированные значения.
for (size_t i = 0; i < query.key_attributes.size(); ++i)
{
const QueryParseResult::KeyAttribute & key = query.key_attributes[i];
std::string s = convertAttributeFormatted(key.attribute, key.parameter, regions_point_of_view_formatted);
if (i > 0)
out_query += ", ";
out_query += s + " AS _" + firstWord(key.attribute) + (key.parameter ? "_" + toString(key.parameter) : "");
selected_expressions.push_back(s);
}
for (size_t i = 0; i < query.aggregates.size(); ++i)
{
const QueryParseResult::Aggregate & aggregate = query.aggregates[i];
std::string s = convertAggregateFunction(aggregate.attribute, aggregate.parameter, aggregate.function, query, regions_point_of_view_formatted);
if (query.key_attributes.size() + i > 0)
out_query += ", ";
out_query += s + " AS _" + firstWord(aggregate.function) + "_" + firstWord(aggregate.attribute) + (aggregate.parameter ? "_" + toString(aggregate.parameter) : "");
selected_expressions.push_back(s);
}
/// Из какой таблицы.
out_query += " FROM " + getTableName(query.CounterID, query.local);
/// Добавляем сэмплирование.
if (query.sample != 1)
out_query += " SAMPLE " + toString(query.sample);
/// Условия.
out_query += " WHERE ";
/// Диапазон дат.
out_query += convertDateRange(query.date_first, query.date_last);
/// Счетчик.
if (query.CounterID != 0)
out_query += " AND " + convertCounterID(query.CounterID);
/// Произвольные условия.
for (size_t i = 0; i < query.where_conditions.size(); ++i)
{
......@@ -131,7 +140,7 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query,
out_query += " AND " + convertCondition(
condition.attribute, condition.parameter, condition.relation, condition.rhs, regions_point_of_view_formatted);
}
/// Группировка.
if (!query.key_attributes.empty())
{
......@@ -143,10 +152,10 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query,
out_query += selected_expressions[i];
}
}
/// Условие для групп.
out_query += " " + getHavingSection();
/// Сортировка.
if (!query.sort_columns.empty())
{
......@@ -154,14 +163,14 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query,
for (size_t i = 0; i < query.sort_columns.size(); ++i)
{
const QueryParseResult::SortColumn & column = query.sort_columns[i];
if (i > 0)
out_query += ", ";
out_query += selected_expressions[column.index - 1];
out_query += " " + convertSortDirection(column.direction);
}
}
/// Ограничение на количество выводимых строк.
if (query.limit != 0)
out_query += " LIMIT " + toString(query.limit);
......@@ -172,7 +181,7 @@ std::string QueryConverter::convertAttributeFormatted(const std::string & attrib
{
if (formatted_attribute_map.count(attribute))
return Poco::format(formatted_attribute_map.at(attribute), parameter);
/** Для атрибутов по регионам, выражение содержит подстановку %s,
* куда должна быть подставлена regions_point_of_view_formatted.
*/
......@@ -182,13 +191,13 @@ std::string QueryConverter::convertAttributeFormatted(const std::string & attrib
if (numeric_attribute_map.count(attribute))
{
std::string numeric = Poco::format(numeric_attribute_map.at(attribute), parameter);
if (formatting_aggregated_attribute_map.count(attribute))
return Poco::format(formatting_aggregated_attribute_map.at(attribute), std::string("(") + numeric + ")");
else
return numeric;
}
throw Exception("Unknown attribute: " + attribute, ErrorCodes::UNKNOWN_IDENTIFIER);
}
......@@ -203,7 +212,7 @@ std::string QueryConverter::convertAttributeNumeric(const std::string & attribut
if (numeric_attribute_map.count(attribute))
return Poco::format(numeric_attribute_map.at(attribute), parameter);
throw Exception("Unknown attribute: " + attribute, ErrorCodes::UNKNOWN_IDENTIFIER);
}
......@@ -228,9 +237,9 @@ std::string QueryConverter::convertAggregateFunction(const std::string & attribu
else
return "sum(Sign)";
}
std::string numeric = convertAttributeNumeric(attribute, parameter, regions_point_of_view_formatted);
if (name == "uniq" ||
name == "uniq_sort" ||
name == "uniq_hash" ||
......@@ -241,13 +250,13 @@ std::string QueryConverter::convertAggregateFunction(const std::string & attribu
if (name == "uniq_state")
return "uniqState(" + numeric + ")";
if (name == "uniq_hll12")
return "uniqHLL12(" + numeric + ")";
if (name == "uniq_hll12_state")
return "uniqHLL12State(" + numeric + ")";
if (name == "count_non_zero")
{
if (query.sample != 1)
......@@ -271,7 +280,7 @@ std::string QueryConverter::convertAggregateFunction(const std::string & attribu
}
bool trivial_format;
std::string format;
if (formatting_aggregated_attribute_map.count(attribute))
{
......@@ -285,7 +294,7 @@ std::string QueryConverter::convertAggregateFunction(const std::string & attribu
}
std::string s;
if (name == "sum")
{
if (query.sample != 1)
......@@ -326,10 +335,10 @@ std::string QueryConverter::convertAggregateFunction(const std::string & attribu
s = "min(" + numeric + ")";
if (name == "max")
s = "max(" + numeric + ")";
/// Если агрегатная функция возвращает дробное число, и атрибут имеет нетривиальное форматирование, после агрегации приведем дробное число к целому.
bool need_cast = !trivial_format && float_value;
return Poco::format(format, std::string() + (need_cast ? "toInt64" : "") + "(" + s + ")");
}
......@@ -349,7 +358,7 @@ std::string QueryConverter::convertCondition(
{
std::string value = convertAttributeNumeric(attribute, parameter, regions_point_of_view_formatted);
std::string constant = convertConstant(attribute, rhs);
if (name == "equals")
return "(" + value + ")" + " == " + constant;
if (name == "not_equals")
......@@ -436,7 +445,7 @@ void QueryConverter::fillNumericAttributeMap()
M("VisitStartWeek", "toInt32(toDateTime(toMonday(StartDate)))")
M("VisitStartTime", "toInt32(toTime(StartTime))")
M("VisitStartTimeRoundedToMinute", "toInt32(toStartOfMinute(toTime(StartTime)))")
M("VisitStartYear", "toYear(StartDate)")
M("VisitStartMonth", "toMonth(StartDate)")
M("VisitStartDayOfWeek", "toDayOfWeek(StartDate)")
......@@ -449,7 +458,7 @@ void QueryConverter::fillNumericAttributeMap()
M("FirstVisitDate", "toInt32(toDateTime(toDate(FirstVisit)))")
M("FirstVisitWeek", "toInt32(toDateTime(toMonday(FirstVisit)))")
M("FirstVisitTime", "toInt32(toTime(FirstVisit))")
M("FirstVisitYear", "toYear(FirstVisit)")
M("FirstVisitMonth", "toMonth(FirstVisit)")
M("FirstVisitDayOfWeek", "toDayOfWeek(FirstVisit)")
......@@ -470,7 +479,7 @@ void QueryConverter::fillNumericAttributeMap()
M("ClientTimeHour", "toHour(ClientEventTime)")
M("ClientTimeMinute", "toMinute(ClientEventTime)")
M("ClientTimeSecond", "toSecond(ClientEventTime)")
M("SearchPhraseHash", "SearchPhraseHash")
M("RefererDomainHash", "RefererDomainHash")
M("StartURLHash", "NormalizedStartURLHash")
......@@ -495,10 +504,10 @@ void QueryConverter::fillNumericAttributeMap()
M("BouncePrecise", "IsBounce")
M("IsYandex", "IsYandex")
M("UserID", "UserID")
M("UserIDCreateDateTime", "(UserID > 10000000000000000000 OR UserID %% 10000000000 > 2000000000 OR UserID %% 10000000000 < 1000000000 ? toUInt64(0) : UserID %% 10000000000)")
M("UserIDCreateDate", "(UserID > 10000000000000000000 OR UserID %% 10000000000 > 2000000000 OR UserID %% 10000000000 < 1000000000 ? toUInt64(0) : UserID %% 10000000000)")
M("UserIDAge", "(UserID > 10000000000000000000 OR UserID %% 10000000000 < 1000000000 OR UserID %% 10000000000 > toUInt64(StartTime) ? toInt64(-1) : intDiv(toInt64(StartTime) - UserID %% 10000000000, 86400))")
M("UserIDAgeInterval", "(UserID > 10000000000000000000 OR UserID %% 10000000000 < 1000000000 OR UserID %% 10000000000 > toUInt64(StartTime) ? toInt64(-1) : toInt64(roundToExp2(intDiv(toUInt64(StartTime) - UserID %% 10000000000, 86400))))")
M("TotalVisits", "TotalVisits")
......@@ -508,18 +517,18 @@ void QueryConverter::fillNumericAttributeMap()
M("Sex", "Sex")
M("Income", "Income")
M("AdvEngineID", "AdvEngineID")
M("DotNet", "NetMajor * 256 + NetMinor")
M("DotNetMajor", "NetMajor")
M("Flash", "FlashMajor * 256 + FlashMinor")
M("FlashExists", "FlashMajor > 0")
M("FlashMajor", "FlashMajor")
M("Silverlight", "SilverlightVersion1 * 72057594037927936 + SilverlightVersion2 * 281474976710656 + SilverlightVersion3 * 65536 + SilverlightVersion4")
M("SilverlightMajor", "SilverlightVersion1")
M("Hits", "Hits")
M("HitsInterval", "roundToExp2(Hits)")
......@@ -529,30 +538,30 @@ void QueryConverter::fillNumericAttributeMap()
M("IsMobile", "IsMobile")
M("MobilePhoneID", "MobilePhone")
M("MobilePhoneModelHash", "halfMD5(MobilePhoneModel)")
M("MobilePhoneModel", "reinterpretAsUInt64(MobilePhoneModel)")
M("BrowserLanguage", "BrowserLanguage")
M("BrowserCountry", "BrowserCountry")
M("TopLevelDomain", "TopLevelDomain")
M("URLScheme", "URLScheme")
M("IPNetworkID", "IPNetworkID")
M("ClientTimeZone", "ClientTimeZone")
M("OSID", "OS")
M("OSMostAncestor", "OSToRoot(OS)")
M("ClientIP", "ClientIP")
M("Resolution", "ResolutionWidth * 16777216 + ResolutionHeight * 256 + ResolutionDepth")
M("ResolutionWidthHeight","ResolutionWidth * 65536 + ResolutionHeight")
M("ResolutionWidth", "ResolutionWidth")
M("ResolutionHeight", "ResolutionHeight")
M("ResolutionWidthInterval","intDiv(ResolutionWidth, 100) * 100")
M("ResolutionHeightInterval","intDiv(ResolutionHeight, 100) * 100")
M("ResolutionColor", "ResolutionDepth")
M("WindowClientArea", "WindowClientWidth * 65536 + WindowClientHeight")
M("WindowClientAreaInterval","intDiv(WindowClientWidth, 100) * 6553600 + intDiv(WindowClientHeight, 100) * 100")
M("WindowClientWidth", "WindowClientWidth")
M("WindowClientWidthInterval","intDiv(WindowClientWidth, 100) * 100")
......@@ -561,11 +570,11 @@ void QueryConverter::fillNumericAttributeMap()
M("SearchEngineID", "SearchEngineID")
M("SearchEngineMostAncestor", "SEToRoot(toUInt8(SearchEngineID))")
M("CodeVersion", "CodeVersion")
M("UserAgent", "UserAgent * 16777216 + UserAgentMajor * 65536 + UserAgentMinor")
M("UserAgentVersion", "UserAgentMajor * 65536 + UserAgentMinor")
M("UserAgentMajor", "UserAgent * 256 + UserAgentMajor")
M("UserAgentID", "UserAgent")
M("ClickGoodEvent", "ClickGoodEvent")
M("ClickPriorityID", "ClickPriorityID")
......@@ -588,9 +597,9 @@ void QueryConverter::fillNumericAttributeMap()
M("Converted", "has(Goals.ID, toUInt32(%u))")
M("CounterID", "CounterID")
M("VisitID", "VisitID")
M("Interests", "Interests")
M("HasInterestPhoto", "modulo(intDiv(Interests, 128), 2)")
M("HasInterestMoviePremieres","modulo(intDiv(Interests, 64), 2)")
M("HasInterestTourism", "modulo(intDiv(Interests, 32), 2)")
......@@ -612,12 +621,12 @@ void QueryConverter::fillNumericAttributeMap()
M("UTMTermHash", "UTMTermHash")
M("FromHash", "FromHash")
M("CLID", "CLID")
M("SocialSourceNetworkID","SocialSourceNetworkID")
/// где 26 это Яндекс (db_dumps/SearchEngines).
M("CorrectedTraficSourceID", "(IsYandex AND SEIn(toUInt8(SearchEngineID), 26)) ? -1 : TraficSourceID")
M("CorrectedSearchEngineID", "(IsYandex AND SEIn(toUInt8(SearchEngineID), 26)) ? 0 : toUInt8(SearchEngineID)")
#undef M
}
......@@ -633,33 +642,33 @@ void QueryConverter::fillFormattedAttributeMap()
M("VisitStartDateTimeRoundedToHour", "toStartOfHour(StartTime)")
M("VisitStartDateRoundedToMonth", "toDateTime(toStartOfMonth(StartDate))")
M("VisitStartTimeRoundedToMinute", "substring(toString(toStartOfMinute(toTime(StartTime))), 12, 8)")
M("FirstVisitDateTime", "FirstVisit")
M("FirstVisitDate", "toDate(FirstVisit)")
M("FirstVisitWeek", "toMonday(FirstVisit)")
M("FirstVisitTime", "substring(toString(FirstVisit), 12, 8)")
M("PredLastVisitDate", "PredLastVisit")
M("PredLastVisitWeek", "toMonday(PredLastVisit)")
M("ClientDateTime", "ClientEventTime")
M("ClientTime", "substring(toString(ClientEventTime), 12, 8)")
M("DotNet", "concat(concat(toString(NetMajor), '.'), toString(NetMinor))")
M("Flash", "concat(concat(toString(FlashMajor),'.'),toString(FlashMinor))")
M("Silverlight", "concat(concat(concat(concat(concat(concat(toString(SilverlightVersion1), '.'), toString(SilverlightVersion2)), '.'), toString(SilverlightVersion3)), '.'), toString(SilverlightVersion4))")
M("MobilePhoneModel", "MobilePhoneModel")
M("ClientIP", "IPv4NumToString(ClientIP)")
M("Resolution", "concat(concat(concat(concat(toString(ResolutionWidth),'x'),toString(ResolutionHeight)),'x'),toString(ResolutionDepth))")
M("ResolutionWidthHeight","concat(concat(toString(ResolutionWidth),'x'),toString(ResolutionHeight))")
M("WindowClientArea", "concat(concat(toString(WindowClientWidth),'x'),toString(WindowClientHeight))")
M("UserAgent", "concat(concat(concat(toString(UserAgent), ' '), toString(UserAgentMajor)), UserAgentMinor == 0 ? '' : concat('.', reinterpretAsString(UserAgentMinor)))")
M("UserAgentVersion", "concat(toString(UserAgentMajor), UserAgentMinor == 0 ? '' : concat('.', reinterpretAsString(UserAgentMinor)))")
M("UserAgentMajor", "concat(concat(toString(UserAgent), ' '), toString(UserAgentMajor))")
#undef M
#undef M
}
void QueryConverter::fillFormattingAggregatedAttributeMap()
......
......@@ -16,6 +16,7 @@ namespace DB
extern const int NOT_FOUND_NODE;
extern const int FOUND_MORE_THAN_ONE_NODE;
extern const int SYNTAX_ERROR;
extern const int UNKNOWN_FORMAT;
extern const int FIRST_DATE_IS_BIGGER_THAN_LAST_DATE;
extern const int UNKNOWN_OVERFLOW_MODE;
extern const int NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册