Small refinements [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-06-01 16:41:58 +03:00
parent 3bb0a07e17
commit 862a304db2
17 changed files with 195 additions and 212 deletions

View File

@ -6,7 +6,8 @@
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
#include <Poco/Semaphore.h> #include <Poco/Semaphore.h>
#include <Core/Types.h> #include <common/Types.h>
namespace detail namespace detail
{ {
@ -66,8 +67,8 @@ public:
fill_count.set(); fill_count.set();
} }
template <class ... Args> template <typename... Args>
void emplace(Args && ... args) void emplace(Args &&... args)
{ {
empty_count.wait(); empty_count.wait();
{ {
@ -88,7 +89,7 @@ public:
empty_count.set(); empty_count.set();
} }
bool tryPush(const T & x, DB::UInt64 milliseconds = 0) bool tryPush(const T & x, UInt64 milliseconds = 0)
{ {
if (empty_count.tryWait(milliseconds)) if (empty_count.tryWait(milliseconds))
{ {
@ -102,8 +103,8 @@ public:
return false; return false;
} }
template <class ... Args> template <typename... Args>
bool tryEmplace(DB::UInt64 milliseconds, Args && ... args) bool tryEmplace(UInt64 milliseconds, Args &&... args)
{ {
if (empty_count.tryWait(milliseconds)) if (empty_count.tryWait(milliseconds))
{ {
@ -117,7 +118,7 @@ public:
return false; return false;
} }
bool tryPop(T & x, DB::UInt64 milliseconds = 0) bool tryPop(T & x, UInt64 milliseconds = 0)
{ {
if (fill_count.tryWait(milliseconds)) if (fill_count.tryWait(milliseconds))
{ {

View File

@ -1,4 +1,4 @@
#include <IO/ReadHelpers.h> #include <Common/hex.h>
#include <Common/StringUtils.h> #include <Common/StringUtils.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
@ -11,8 +11,6 @@ std::string escapeForFileName(const std::string & s)
const char * pos = s.data(); const char * pos = s.data();
const char * end = pos + s.size(); const char * end = pos + s.size();
static const char * hex = "0123456789ABCDEF";
while (pos != end) while (pos != end)
{ {
char c = *pos; char c = *pos;
@ -22,8 +20,8 @@ std::string escapeForFileName(const std::string & s)
else else
{ {
res += '%'; res += '%';
res += hex[c / 16]; res += hexUppercase(c / 16);
res += hex[c % 16]; res += hexUppercase(c % 16);
} }
++pos; ++pos;

View File

@ -1,6 +1,9 @@
#include <Common/hex.h> #include <Common/hex.h>
const char * const char_to_digit_table = ( const char * const hex_digit_to_char_uppercase_table = "0123456789ABCDEF";
const char * const hex_digit_to_char_lowercase_table = "0123456789abcdef";
const char * const hex_char_to_digit_table =
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
@ -16,5 +19,4 @@ const char * const char_to_digit_table = (
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff";
);

View File

@ -1,3 +1,27 @@
#pragma once #pragma once
extern const char * const char_to_digit_table;
/// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly.
extern const char * const hex_digit_to_char_uppercase_table;
extern const char * const hex_digit_to_char_lowercase_table;
inline char hexUppercase(unsigned char c)
{
return hex_digit_to_char_uppercase_table[c];
}
inline char hexLowercase(unsigned char c)
{
return hex_digit_to_char_lowercase_table[c];
}
/// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value.
extern const char * const hex_char_to_digit_table;
inline char unhex(char c)
{
return hex_char_to_digit_table[static_cast<unsigned char>(c)];
}

View File

@ -1,11 +1,12 @@
#include <DataStreams/verbosePrintString.h> #include <DataStreams/verbosePrintString.h>
#include <Common/hex.h>
#include <IO/Operators.h> #include <IO/Operators.h>
namespace DB namespace DB
{ {
void verbosePrintString(BufferBase::Position begin, BufferBase::Position end, WriteBuffer & out) void verbosePrintString(const char * begin, const char * end, WriteBuffer & out)
{ {
if (end == begin) if (end == begin)
{ {
@ -50,10 +51,7 @@ void verbosePrintString(BufferBase::Position begin, BufferBase::Position end, Wr
default: default:
{ {
if (*pos >= 0 && *pos < 32) if (*pos >= 0 && *pos < 32)
{ out << "<0x" << hexUppercase(*pos / 16) << hexUppercase(*pos % 16) << ">";
static const char * hex = "0123456789ABCDEF";
out << "<0x" << hex[*pos / 16] << hex[*pos % 16] << ">";
}
else else
out << *pos; out << *pos;
} }

View File

@ -1,12 +1,14 @@
#pragma once #pragma once
#include <IO/WriteBuffer.h>
namespace DB namespace DB
{ {
class WriteBuffer;
/** Print string in double quotes and with control characters in "<NAME>" form - for output diagnostic info to user. /** Print string in double quotes and with control characters in "<NAME>" form - for output diagnostic info to user.
*/ */
void verbosePrintString(BufferBase::Position begin, BufferBase::Position end, WriteBuffer & out); void verbosePrintString(const char * begin, const char * end, WriteBuffer & out);
} }

View File

@ -37,8 +37,8 @@ namespace ErrorCodes
/** Array functions: /** Array functions:
* *
* array(c1, c2, ...) - create an array of constants. * array(c1, c2, ...) - create an array.
* arrayElement(arr, i) - get the array element by index. * arrayElement(arr, i) - get the array element by index. If index is not constant and out of range - return default value of data type.
* The index begins with 1. Also, the index can be negative - then it is counted from the end of the array. * The index begins with 1. Also, the index can be negative - then it is counted from the end of the array.
* has(arr, x) - whether there is an element x in the array. * has(arr, x) - whether there is an element x in the array.
* indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not. * indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not.
@ -48,8 +48,8 @@ namespace ErrorCodes
* arrayUniq(arr1, arr2, ...) - counts the number of different tuples from the elements in the corresponding positions in several arrays. * arrayUniq(arr1, arr2, ...) - counts the number of different tuples from the elements in the corresponding positions in several arrays.
* *
* arrayEnumerateUniq(arr) * arrayEnumerateUniq(arr)
* - outputs an array parallel to this, where for each element specified * - outputs an array parallel (having same size) to this, where for each element specified
* how much times this element was encountered before among elements with the same value. * how much times this element was encountered before (including this element) among elements with the same value.
* For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1] * For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
* arrayEnumerateUniq(arr1, arr2...) * arrayEnumerateUniq(arr1, arr2...)
* - for tuples from elements in the corresponding positions in several arrays. * - for tuples from elements in the corresponding positions in several arrays.
@ -57,6 +57,7 @@ namespace ErrorCodes
* emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value. * emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value.
* *
* arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...` * arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...`
* If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function.
*/ */
@ -193,7 +194,7 @@ private:
static bool hasNull(const U & value, const PaddedPODArray<UInt8> & null_map, size_t i) static bool hasNull(const U & value, const PaddedPODArray<UInt8> & null_map, size_t i)
{ {
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; throw Exception{"Logical error: constant column cannot have null map.", ErrorCodes::LOGICAL_ERROR};
} }
/// Both function arguments are ordinary. /// Both function arguments are ordinary.
@ -1033,7 +1034,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{ {
/// If one or both arguments passed to this function are nullable, /// If one or both arguments passed to this function are nullable,
/// we create a new block that contains non-nullable parameters: /// we create a new block that contains non-nullable arguments:
/// - if the 1st argument is a non-constant array of nullable values, /// - if the 1st argument is a non-constant array of nullable values,
/// it is turned into a non-constant array of ordinary values + a null /// it is turned into a non-constant array of ordinary values + a null
/// byte map; /// byte map;
@ -1199,7 +1200,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private: private:
/// Initially allocate a piece of memory for 512 elements. /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
static constexpr size_t INITIAL_SIZE_DEGREE = 9; static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T> template <typename T>
@ -1239,7 +1240,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private: private:
/// Initially allocate a piece of memory for 512 elements. /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
static constexpr size_t INITIAL_SIZE_DEGREE = 9; static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T> template <typename T>
@ -1382,7 +1383,7 @@ class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>; using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
/** Applies an aggregate function to array and returns its result. /** Applies an aggregate function to array and returns its result.
* If aggregate function has multiple arguments, then this function can be applied to multiple arrays with the same size. * If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size.
*/ */
class FunctionArrayReduce : public IFunction class FunctionArrayReduce : public IFunction
{ {
@ -1406,9 +1407,9 @@ private:
}; };
struct NameHas { static constexpr auto name = "has"; }; struct NameHas { static constexpr auto name = "has"; };
struct NameIndexOf { static constexpr auto name = "indexOf"; }; struct NameIndexOf { static constexpr auto name = "indexOf"; };
struct NameCountEqual { static constexpr auto name = "countEqual"; }; struct NameCountEqual { static constexpr auto name = "countEqual"; };
using FunctionHas = FunctionArrayIndex<IndexToOne, NameHas>; using FunctionHas = FunctionArrayIndex<IndexToOne, NameHas>;
using FunctionIndexOf = FunctionArrayIndex<IndexIdentity, NameIndexOf>; using FunctionIndexOf = FunctionArrayIndex<IndexIdentity, NameIndexOf>;

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <Common/hex.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
@ -24,25 +24,22 @@
namespace DB namespace DB
{ {
/** Coding functions: /** Encoding functions:
* *
* IPv4NumToString (num) - See below. * IPv4NumToString (num) - See below.
* IPv4StringToNum(string) - Convert, for example, '192.168.0.1' to 3232235521 and vice versa. * IPv4StringToNum(string) - Convert, for example, '192.168.0.1' to 3232235521 and vice versa.
* *
* hex(x) - Returns hex; capital letters; there are no prefixes 0x or suffixes h. * hex(x) - Returns hex; capital letters; there are no prefixes 0x or suffixes h.
* For numbers, returns a variable-length string - hex in the "human" (big endian) format, with the leading zeros being cut, * For numbers, returns a variable-length string - hex in the "human" (big endian) format, with the leading zeros being cut,
* but only by whole bytes. For dates and datetimes - the same as for numbers. * but only by whole bytes. For dates and datetimes - the same as for numbers.
* For example, hex(257) = '0101'. * For example, hex(257) = '0101'.
* unhex(string) - Returns a string, hex of which is equal to `string` with regard of case and discarding one leading zero. * unhex(string) - Returns a string, hex of which is equal to `string` with regard of case and discarding one leading zero.
* If such a string does not exist, reserves the right to return any garbage. * If such a string does not exist, could return arbitary implementation specific value.
* *
* bitmaskToArray(x) - Returns an array of powers of two in the binary form of x. For example, bitmaskToArray(50) = [2, 16, 32]. * bitmaskToArray(x) - Returns an array of powers of two in the binary form of x. For example, bitmaskToArray(50) = [2, 16, 32].
*/ */
/// Including zero character at the end.
#define MAX_UINT_HEX_LENGTH 20
const auto ipv4_bytes_length = 4; const auto ipv4_bytes_length = 4;
const auto ipv6_bytes_length = 16; const auto ipv6_bytes_length = 16;
const auto uuid_bytes_length = 16; const auto uuid_bytes_length = 16;
@ -57,9 +54,6 @@ private:
return value >= base ? 1 + int_log(value / base, base, value % base || carry) : value % base > 1 || carry; return value >= base ? 1 + int_log(value / base, base, value % base || carry) : value % base > 1 || carry;
} }
/// mapping of digits up to base 16
static constexpr auto && digits = "0123456789abcdef";
/// print integer in desired base, faster than sprintf /// print integer in desired base, faster than sprintf
template <uint32_t base, typename T, uint32_t buffer_size = sizeof(T) * int_log(256, base, false)> template <uint32_t base, typename T, uint32_t buffer_size = sizeof(T) * int_log(256, base, false)>
static void print_integer(char *& out, T value) static void print_integer(char *& out, T value)
@ -73,7 +67,7 @@ private:
while (value > 0) while (value > 0)
{ {
*ptr++ = digits[value % base]; *ptr++ = hexLowercase(value % base);
value /= base; value /= base;
} }
@ -1013,9 +1007,6 @@ public:
{ {
char * begin = out; char * begin = out;
/// mapping of digits up to base 16
static char digits[] = "0123456789ABCDEF";
/// Write everything backwards. /// Write everything backwards.
for (size_t offset = 0; offset <= 40; offset += 8) for (size_t offset = 0; offset <= 40; offset += 8)
{ {
@ -1038,7 +1029,7 @@ public:
{ {
while (value > 0) while (value > 0)
{ {
*(out++) = digits[value % 16]; *(out++) = hexUppercase(value % 16);
value /= 16; value /= 16;
} }
} }
@ -1596,7 +1587,6 @@ public:
template <typename T> template <typename T>
void executeOneUInt(T x, char *& out) void executeOneUInt(T x, char *& out)
{ {
const char digit[17] = "0123456789ABCDEF";
bool was_nonzero = false; bool was_nonzero = false;
for (int offset = (sizeof(T) - 1) * 8; offset >= 0; offset -= 8) for (int offset = (sizeof(T) - 1) * 8; offset >= 0; offset -= 8)
{ {
@ -1608,8 +1598,8 @@ public:
was_nonzero = true; was_nonzero = true;
*(out++) = digit[byte >> 4]; *(out++) = hexUppercase(byte / 16);
*(out++) = digit[byte & 15]; *(out++) = hexUppercase(byte % 16);
} }
*(out++) = '\0'; *(out++) = '\0';
} }
@ -1620,6 +1610,8 @@ public:
const ColumnVector<T> * col_vec = typeid_cast<const ColumnVector<T> *>(col); const ColumnVector<T> * col_vec = typeid_cast<const ColumnVector<T> *>(col);
const ColumnConst<T> * col_const = typeid_cast<const ColumnConst<T> *>(col); const ColumnConst<T> * col_const = typeid_cast<const ColumnConst<T> *>(col);
static constexpr size_t MAX_UINT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte.
if (col_vec) if (col_vec)
{ {
auto col_str = std::make_shared<ColumnString>(); auto col_str = std::make_shared<ColumnString>();
@ -1631,7 +1623,7 @@ public:
size_t size = in_vec.size(); size_t size = in_vec.size();
out_offsets.resize(size); out_offsets.resize(size);
out_vec.resize(size * 3 + MAX_UINT_HEX_LENGTH); out_vec.resize(size * 3 + MAX_UINT_HEX_LENGTH); /// 3 is length of one byte in hex plus zero byte.
size_t pos = 0; size_t pos = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
@ -1670,12 +1662,11 @@ public:
void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out) void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out)
{ {
const char digit[17] = "0123456789ABCDEF";
while (pos < end) while (pos < end)
{ {
UInt8 byte = *(pos++); UInt8 byte = *(pos++);
*(out++) = digit[byte >> 4]; *(out++) = hexUppercase(byte / 16);
*(out++) = digit[byte & 15]; *(out++) = hexUppercase(byte % 16);
} }
*(out++) = '\0'; *(out++) = '\0';
} }
@ -1719,7 +1710,7 @@ public:
return true; return true;
} }
else if(col_const_in) else if (col_const_in)
{ {
const std::string & src = col_const_in->getData(); const std::string & src = col_const_in->getData();
std::string res(src.size() * 2, '\0'); std::string res(src.size() * 2, '\0');
@ -1829,28 +1820,26 @@ public:
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
UInt8 undigitUnsafe(char c)
{
if (c <= '9')
return c - '0';
if (c <= 'Z')
return c - ('A' - 10);
return c - ('a' - 10);
}
void unhexOne(const char * pos, const char * end, char *& out) void unhexOne(const char * pos, const char * end, char *& out)
{ {
if ((end - pos) & 1) if ((end - pos) & 1)
{ {
*(out++) = undigitUnsafe(*(pos++)); *out = unhex(*pos);
++out;
++pos;
} }
while (pos < end) while (pos < end)
{ {
UInt8 major = undigitUnsafe(*(pos++)); UInt8 major = unhex(*pos);
UInt8 minor = undigitUnsafe(*(pos++)); ++pos;
*(out++) = (major << 4) | minor; UInt8 minor = unhex(*pos);
++pos;
*out = (major << 4) | minor;
++out;
} }
*(out++) = '\0'; *out = '\0';
++out;
} }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override

View File

@ -1001,32 +1001,32 @@ public:
const DataTypeTuple * right_tuple = nullptr; const DataTypeTuple * right_tuple = nullptr;
false false
|| (right_is_date = typeid_cast<const DataTypeDate *>(arguments[1].get())) || (right_is_date = typeid_cast<const DataTypeDate *>(arguments[1].get()))
|| (right_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[1].get())) || (right_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[1].get()))
|| (right_is_enum8 = typeid_cast<const DataTypeEnum8 *>(arguments[1].get())) || (right_is_enum8 = typeid_cast<const DataTypeEnum8 *>(arguments[1].get()))
|| (right_is_enum16 = typeid_cast<const DataTypeEnum16 *>(arguments[1].get())) || (right_is_enum16 = typeid_cast<const DataTypeEnum16 *>(arguments[1].get()))
|| (right_is_string = typeid_cast<const DataTypeString *>(arguments[1].get())) || (right_is_string = typeid_cast<const DataTypeString *>(arguments[1].get()))
|| (right_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[1].get())) || (right_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[1].get()))
|| (right_tuple = typeid_cast<const DataTypeTuple *>(arguments[1].get())); || (right_tuple = typeid_cast<const DataTypeTuple *>(arguments[1].get()));
const bool right_is_enum = right_is_enum8 || right_is_enum16; const bool right_is_enum = right_is_enum8 || right_is_enum16;
if (!( (arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber() && !(left_is_enum ^ right_is_enum)) if (!((arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber() && !(left_is_enum ^ right_is_enum))
|| ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string)) || ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string))
|| (left_is_date && right_is_date) || (left_is_date && right_is_date)
|| (left_is_date && right_is_string) /// You can compare the date, datetime and an enumeration with a constant string. || (left_is_date && right_is_string) /// You can compare the date, datetime and an enumeration with a constant string.
|| (left_is_string && right_is_date) || (left_is_string && right_is_date)
|| (left_is_date_time && right_is_date_time) || (left_is_date_time && right_is_date_time)
|| (left_is_date_time && right_is_string) || (left_is_date_time && right_is_string)
|| (left_is_string && right_is_date_time) || (left_is_string && right_is_date_time)
|| (left_is_date_time && right_is_date_time) || (left_is_date_time && right_is_date_time)
|| (left_is_date_time && right_is_string) || (left_is_date_time && right_is_string)
|| (left_is_string && right_is_date_time) || (left_is_string && right_is_date_time)
|| (left_is_enum && right_is_enum && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against || (left_is_enum && right_is_enum && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against
|| (left_is_enum && right_is_string) || (left_is_enum && right_is_string)
|| (left_is_string && right_is_enum) || (left_is_string && right_is_enum)
|| (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size())
|| (arguments[0]->equals(*arguments[1])))) || (arguments[0]->equals(*arguments[1]))))
throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")" throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")"
" of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); " of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1052,16 +1052,16 @@ public:
if (left_is_num && right_is_num) if (left_is_num && right_is_num)
{ {
if (!( executeNumLeftType<UInt8>(block, result, col_left_untyped, col_right_untyped) if (!( executeNumLeftType<UInt8>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<UInt16>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<UInt16>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<UInt32>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<UInt32>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<UInt64>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<UInt64>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Int8>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<Int8>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Int16>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<Int16>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Int32>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<Int32>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Int64>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<Int64>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Float32>(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType<Float32>(block, result, col_left_untyped, col_right_untyped)
|| executeNumLeftType<Float64>(block, result, col_left_untyped, col_right_untyped))) || executeNumLeftType<Float64>(block, result, col_left_untyped, col_right_untyped)))
throw Exception("Illegal column " + col_left_untyped->getName() throw Exception("Illegal column " + col_left_untyped->getName()
+ " of first argument of function " + getName(), + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
@ -1077,15 +1077,15 @@ public:
block, result, col_left_untyped, col_right_untyped, block, result, col_left_untyped, col_right_untyped,
col_with_type_and_name_left.type, col_with_type_and_name_right.type, col_with_type_and_name_left.type, col_with_type_and_name_right.type,
left_is_num, right_is_num); left_is_num, right_is_num);
} }
}; };
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals> ; using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
using FunctionNotEquals = FunctionComparison<NotEqualsOp, NameNotEquals> ; using FunctionNotEquals = FunctionComparison<NotEqualsOp, NameNotEquals>;
using FunctionLess = FunctionComparison<LessOp, NameLess> ; using FunctionLess = FunctionComparison<LessOp, NameLess>;
using FunctionGreater = FunctionComparison<GreaterOp, NameGreater> ; using FunctionGreater = FunctionComparison<GreaterOp, NameGreater>;
using FunctionLessOrEquals = FunctionComparison<LessOrEqualsOp, NameLessOrEquals> ; using FunctionLessOrEquals = FunctionComparison<LessOrEqualsOp, NameLessOrEquals>;
using FunctionGreaterOrEquals = FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>; using FunctionGreaterOrEquals = FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>;
} }

View File

@ -20,7 +20,9 @@ static size_t decodeURL(const char * src, size_t src_size, char * dst)
src_curr_pos = find_first_symbols<'%'>(src_curr_pos, src_end); src_curr_pos = find_first_symbols<'%'>(src_curr_pos, src_end);
if (src_curr_pos == src_end) if (src_curr_pos == src_end)
{
break; break;
}
else if (src_end - src_curr_pos < 3) else if (src_end - src_curr_pos < 3)
{ {
src_curr_pos = src_end; src_curr_pos = src_end;
@ -28,8 +30,8 @@ static size_t decodeURL(const char * src, size_t src_size, char * dst)
} }
else else
{ {
unsigned char high = char_to_digit_table[static_cast<unsigned char>(src_curr_pos[1])]; unsigned char high = unhex(src_curr_pos[1]);
unsigned char low = char_to_digit_table[static_cast<unsigned char>(src_curr_pos[2])]; unsigned char low = unhex(src_curr_pos[2]);
if (high != 0xFF && low != 0xFF) if (high != 0xFF && low != 0xFF)
{ {

View File

@ -10,14 +10,17 @@
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h> #include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Common/hex.h>
#include <Common/Volnitsky.h> #include <Common/Volnitsky.h>
#include <Functions/IFunction.h> #include <Functions/IFunction.h>
#include <IO/ReadBufferFromMemory.h> #include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
/** Functions for retrieving visit parameters. /** Functions for retrieving "visit parameters".
* Implemented via templates from FunctionsStringSearch.h. * Visit parameters in Yandex.Metrika are a special kind of JSONs.
* These functions are applicable to almost any JSONs.
* Implemented via templates from FunctionsStringSearch.h.
* *
* Check if there is a parameter * Check if there is a parameter
* visitParamHas * visitParamHas
@ -140,50 +143,12 @@ struct ExtractRaw
struct ExtractString struct ExtractString
{ {
static bool tryParseDigit(UInt8 c, UInt8 & res) static UInt64 unhexCodePoint(const UInt8 * pos)
{ {
if ('0' <= c && c <= '9') return unhex(pos[0]) * 0xFFF
{ + unhex(pos[1]) * 0xFF
res = c - '0'; + unhex(pos[2]) * 0xF
return true; + unhex(pos[3]);
}
if ('A' <= c && c <= 'Z')
{
res = c - ('A' - 10);
return true;
}
if ('a' <= c && c <= 'z')
{
res = c - ('a' - 10);
return true;
}
return false;
}
static bool tryUnhex(const UInt8 * pos, const UInt8 * end, int & res)
{
if (pos + 3 >= end)
return false;
res = 0;
{
UInt8 major, minor;
if (!tryParseDigit(*(pos++), major))
return false;
if (!tryParseDigit(*(pos++), minor))
return false;
res |= (major << 4) | minor;
}
res <<= 8;
{
UInt8 major, minor;
if (!tryParseDigit(*(pos++), major))
return false;
if (!tryParseDigit(*(pos++), minor))
return false;
res |= (major << 4) | minor;
}
return true;
} }
static bool tryExtract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars_t & res_data) static bool tryExtract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars_t & res_data)
@ -231,20 +196,25 @@ struct ExtractString
{ {
++pos; ++pos;
int unicode; if (pos + 4 > end)
if (!tryUnhex(pos, end, unicode))
return false; return false;
UInt16 code_point = unhexCodePoint(pos);
pos += 3; pos += 3;
res_data.resize(res_data.size() + 6); /// the maximum size of the UTF8 multibyte sequence static constexpr size_t max_code_point_byte_length = 4;
size_t old_size = res_data.size();
res_data.resize(old_size + max_code_point_byte_length);
Poco::UTF8Encoding utf8; Poco::UTF8Encoding utf8;
int length = utf8.convert(unicode, const_cast<UInt8 *>(&res_data[0]) + res_data.size() - 6, 6); int length = utf8.convert(code_point,
&res_data[old_size], max_code_point_byte_length);
if (!length) if (!length)
return false; return false;
res_data.resize(res_data.size() - 6 + length); res_data.resize(old_size + length);
break; break;
} }
default: default:

View File

@ -0,0 +1,35 @@
#include <Core/Types.h>
#include <Common/hex.h>
#include <Common/Exception.h>
#include <IO/HexWriteBuffer.h>
namespace DB
{
void HexWriteBuffer::nextImpl()
{
if (!offset())
return;
for (Position p = working_buffer.begin(); p != pos; ++p)
{
UInt8 byte = *p;
out.write(hexUppercase(byte / 16));
out.write(hexUppercase(byte % 16));
}
}
HexWriteBuffer::~HexWriteBuffer()
{
try
{
nextImpl();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}

View File

@ -1,6 +1,5 @@
#pragma once #pragma once
#include <IO/BufferWithOwnMemory.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
@ -13,39 +12,17 @@ namespace DB
/** Всё что в него пишут, переводит в HEX (большими буквами) и пишет в другой WriteBuffer. /** Всё что в него пишут, переводит в HEX (большими буквами) и пишет в другой WriteBuffer.
*/ */
class HexWriteBuffer : public WriteBuffer class HexWriteBuffer final : public WriteBuffer
{ {
protected: protected:
char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; char buf[DBMS_HEX_WRITE_BUFFER_SIZE];
WriteBuffer & out; WriteBuffer & out;
void nextImpl() override void nextImpl() override;
{
if (!offset())
return;
for (Position p = working_buffer.begin(); p != pos; ++p)
{
out.write("0123456789ABCDEF"[static_cast<unsigned char>(*p) >> 4]);
out.write("0123456789ABCDEF"[static_cast<unsigned char>(*p) & 0xF]);
}
}
public: public:
HexWriteBuffer(WriteBuffer & out_) : WriteBuffer(buf, sizeof(buf)), out(out_) {} HexWriteBuffer(WriteBuffer & out_) : WriteBuffer(buf, sizeof(buf)), out(out_) {}
~HexWriteBuffer() override;
~HexWriteBuffer() override
{
try
{
nextImpl();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}; };
} }

View File

@ -1,4 +1,5 @@
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Common/hex.h>
#include <Common/PODArray.h> #include <Common/PODArray.h>
#include <Common/StringUtils.h> #include <Common/StringUtils.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -64,21 +64,6 @@ inline char parseEscapeSequence(char c)
} }
} }
inline char unhex(char c)
{
switch (c)
{
case '0' ... '9':
return c - '0';
case 'a' ... 'f':
return c - 'a' + 10;
case 'A' ... 'F':
return c - 'A' + 10;
default:
return 0;
}
}
/// These functions are located in VarInt.h /// These functions are located in VarInt.h
/// inline void throwReadAfterEOF() /// inline void throwReadAfterEOF()

View File

@ -552,9 +552,6 @@
Хотя, при прочих равных условиях, предпочитается более-менее кроссплатформенный или легко портируемый код. Хотя, при прочих равных условиях, предпочитается более-менее кроссплатформенный или легко портируемый код.
2. Язык - C++17. Возможно использование расширений GNU при необходимости. 2. Язык - C++17. Возможно использование расширений GNU при необходимости.
```cpp
case '0' ... '9':
```
3. Компилятор - gcc. На данный момент (апрель 2017), код собирается версией 6.3. (Также код может быть собран clang 4) 3. Компилятор - gcc. На данный момент (апрель 2017), код собирается версией 6.3. (Также код может быть собран clang 4)
Используется стандартная библиотека от gcc. Используется стандартная библиотека от gcc.

View File

@ -1,5 +1,6 @@
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
#include <Common/hex.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
@ -92,7 +93,7 @@ static void readPath(std::string & s, DB::ReadBuffer & buf)
s += c2; s += c2;
} }
else else
s += static_cast<char>(static_cast<UInt8>(DB::unhex(c1)) * 16 + static_cast<UInt8>(DB::unhex(c2))); s += static_cast<char>(static_cast<UInt8>(unhex(c1)) * 16 + static_cast<UInt8>(unhex(c2)));
} }
} }
} }