Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2017-05-30 11:58:37 +03:00
commit 5312f9b7bf
371 changed files with 4782 additions and 2785 deletions

View File

@ -1,5 +1,3 @@
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
[Read more...](https://clickhouse.yandex/)
[Changelog](CHANGELOG.md)

View File

@ -22,7 +22,7 @@ PATH="/usr/local/bin:/usr/local/sbin:/usr/bin:$PATH"
# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом).
for src_file in $(clang -M -xc++ -std=gnu++1y -Wall -Werror -msse4 -mcx16 -mpopcnt -O3 -g -fPIC \
for src_file in $($CLANG -M -xc++ -std=gnu++1y -Wall -Werror -msse4 -mcx16 -mpopcnt -O3 -g -fPIC \
$(cat "$SOURCE_PATH/build/include_directories.txt") \
"$SOURCE_PATH/dbms/src/Interpreters/SpecializedAggregator.h" |
tr -d '\\' |
@ -37,7 +37,7 @@ done
# Копируем больше заголовочных файлов с интринсиками, так как на серверах, куда будут устанавливаться
# заголовочные файлы, будет использоваться опция -march=native.
for i in $(ls -1 $(clang -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep '/lib/clang/')/*.h | grep -vE 'arm|altivec|Intrin');
for i in $(ls -1 $($CLANG -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep '/lib/clang/')/*.h | grep -vE 'arm|altivec|Intrin');
do
cp "$i" "$DST/$i";
done

View File

@ -1,6 +1,6 @@
#This strings autochanged from release_lib.sh :
set(VERSION_DESCRIBE v1.1.54235-testing)
set(VERSION_REVISION 54235)
set(VERSION_DESCRIBE v1.1.54236-testing)
set(VERSION_REVISION 54236)
#===end of autochange
set (VERSION_MAJOR 1)

View File

@ -216,7 +216,7 @@ struct StdDevPopImpl
}
/** If `compute_marginal_moments` flag is set this class provides the heir
/** If `compute_marginal_moments` flag is set this class provides the successor
* CovarianceData support of marginal moments for calculating the correlation.
*/
template<bool compute_marginal_moments>

View File

@ -20,7 +20,7 @@ struct CollectTables;
*
* SELECT elem FROM t ARRAY JOIN array AS elem elem -> array
* SELECT n.elem FROM t ARRAY JOIN nested AS n n -> nested
* SELECT array FROM t ARRAY JOIN array array -> array
* SELECT array FROM t ARRAY JOIN array array -> array
* SELECT nested.elem FROM t ARRAY JOIN nested nested -> nested
* SELECT elem FROM t ARRAY JOIN [1, 2, 3] AS elem elem -> [1, 2, 3]
*/

View File

@ -379,8 +379,17 @@ void processImpl(ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectA
void AnalyzeColumns::process(ASTPtr & ast, const CollectAliases & aliases, const CollectTables & tables)
{
/// If this is SELECT query, don't go into FORMAT and SETTINGS clauses
/// - they contain identifiers that are not columns.
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
for (auto & child : ast->children)
{
if (select && (child.get() == select->format.get() || child.get() == select->settings.get()))
continue;
processImpl(child, columns, aliases, tables);
}
}

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
}
void AnalyzeResultOfQuery::process(ASTPtr & ast, Context & context)
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)

View File

@ -19,7 +19,7 @@ class Context;
*/
struct AnalyzeResultOfQuery
{
void process(ASTPtr & ast, Context & context);
void process(ASTPtr & ast, const Context & context);
/// Block will have non-nullptr columns for constant expressions.
Block result;

View File

@ -31,9 +31,9 @@ struct CollectAliases
enum class Kind
{
Expression, /// Example: SELECT a AS b, f(x) AS y
Table, /// Example: SELECT t.* FROM (SELECT 1) AS t
ArrayJoin /// Example: SELECT t.x.a FROM t ARRAY JOIN arr AS x
Expression, /// Example: SELECT a AS b, f(x) AS y
Table, /// Example: SELECT t.* FROM (SELECT 1) AS t
ArrayJoin /// Example: SELECT t.x.a FROM t ARRAY JOIN arr AS x
};
struct AliasInfo

View File

@ -49,7 +49,7 @@ static CollectTables::TableInfo processOrdinaryTable(const ASTPtr & ast_database
}
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, Context & context)
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const Context & context)
{
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
@ -78,7 +78,7 @@ static CollectTables::TableInfo processNoTables(const Context & context)
}
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, Context & context)
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context)
{
AnalyzeResultOfQuery analyzer;
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context);
@ -91,7 +91,7 @@ static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, Context &
}
void CollectTables::process(ASTPtr & ast, Context & context, const CollectAliases & aliases)
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)

View File

@ -24,7 +24,7 @@ class WriteBuffer;
*/
struct CollectTables
{
void process(ASTPtr & ast, Context & context, const CollectAliases & aliases);
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases);
enum class Kind
{

View File

@ -56,7 +56,7 @@ using LambdaParameters = std::unordered_map<String, DataTypePtr>;
void processImpl(
ASTPtr & ast, Context & context,
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas);
@ -76,7 +76,7 @@ void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConst
void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info,
Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
const AnalyzeLambdas & lambdas)
{
/// Column from table
@ -258,7 +258,7 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
Context & context)
const Context & context)
{
ASTSubquery * subquery = static_cast<ASTSubquery *>(ast.get());
@ -316,7 +316,7 @@ void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndCons
void processHigherOrderFunction(const String & column_name,
ASTPtr & ast, Context & context,
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas)
@ -395,7 +395,7 @@ void processHigherOrderFunction(const String & column_name,
void processImpl(
ASTPtr & ast, Context & context,
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas)
@ -468,7 +468,7 @@ void processImpl(
}
void TypeAndConstantInference::process(ASTPtr & ast, Context & context,
void TypeAndConstantInference::process(ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns, const AnalyzeLambdas & lambdas)
{
processImpl(ast, context, aliases, columns, info, lambdas);

View File

@ -30,7 +30,7 @@ class IAggregateFunction;
*/
struct TypeAndConstantInference
{
void process(ASTPtr & ast, Context & context,
void process(ASTPtr & ast, const Context & context,
CollectAliases & aliases,
const AnalyzeColumns & columns,
const AnalyzeLambdas & analyze_lambdas);

View File

@ -1,4 +1,4 @@
#include <numeric>
#include <Core/NamesAndTypes.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -137,10 +137,9 @@ static ASTPtr buildWhereExpression(const ASTs & functions)
return new_query;
}
bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context)
{
query = query->clone();
const ASTSelectQuery & select = typeid_cast<ASTSelectQuery & >(*query);
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery & >(*query);
if (!select.where_expression && !select.prewhere_expression)
return false;
@ -170,7 +169,7 @@ bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
filter_column = converted;
const IColumn::Filter & filter = dynamic_cast<ColumnUInt8 &>(*filter_column).getData();
if (std::accumulate(filter.begin(), filter.end(), 0ul) == filter.size())
if (countBytesInFilter(filter) == 0)
return false;
for (size_t i = 0; i < block.columns(); ++i)

View File

@ -3,7 +3,6 @@
#include <set>
#include <Core/Block.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/IAST.h>
@ -11,6 +10,7 @@ namespace DB
{
class Context;
class NamesAndTypesList;
namespace VirtualColumnUtils
@ -30,10 +30,10 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va
/// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query.
/// Only elements of the outer conjunction are considered, depending only on the columns present in the block.
/// Returns true if at least one row is discarded.
bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context);
bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context);
/// Extract from the input stream a set of `name` column values
template<typename T1>
template <typename T1>
std::multiset<T1> extractSingleValueFromBlock(const Block & block, const String & name)
{
std::multiset<T1> res;

View File

@ -24,13 +24,23 @@ unsigned getNumberOfPhysicalCPUCores()
throw DB::Exception("Cannot cpu_identify: " + std::string(cpuid_error()), DB::ErrorCodes::CPUID_ERROR);
/// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method.
if (data.num_cores == 0 || data.total_logical_cpus == 0 || data.num_logical_cpus == 0)
if (data.num_logical_cpus == 0)
return std::thread::hardware_concurrency();
return data.num_cores * data.total_logical_cpus / data.num_logical_cpus;
unsigned res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus;
#elif defined(__aarch64__)
/// Assuming there are no hyper-threading on the system.
return std::thread::hardware_concurrency();
/// Also, libcpuid gives strange result on Google Compute Engine VMs.
/// Example:
/// num_cores = 12, /// number of physical cores on current CPU socket
/// total_logical_cpus = 1, /// total number of logical cores on all sockets
/// num_logical_cpus = 24. /// number of logical cores on current CPU socket
/// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1.
if (res != 0)
return res;
#endif
/// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system.
/// (Actually, only Aarch64 is supported).
return std::thread::hardware_concurrency();
}

View File

@ -30,12 +30,12 @@ using DB::UInt64;
// Case 1. Is pair of floats or pair of ints or pair of uints
template <typename A, typename B>
using is_safe_convervsion = std::integral_constant<bool, (std::is_floating_point<A>::value && std::is_floating_point<B>::value)
using is_safe_conversion = std::integral_constant<bool, (std::is_floating_point<A>::value && std::is_floating_point<B>::value)
|| (std::is_integral<A>::value && std::is_integral<B>::value && !(std::is_signed<A>::value ^ std::is_signed<B>::value))>;
template <typename A, typename B>
using bool_if_safe_convervsion = std::enable_if_t<is_safe_convervsion<A, B>::value, bool>;
using bool_if_safe_conversion = std::enable_if_t<is_safe_conversion<A, B>::value, bool>;
template <typename A, typename B>
using bool_if_not_safe_convervsion = std::enable_if_t<!is_safe_convervsion<A, B>::value, bool>;
using bool_if_not_safe_conversion = std::enable_if_t<!is_safe_conversion<A, B>::value, bool>;
/// Case 2. Are params IntXX and UIntYY ?
@ -144,13 +144,13 @@ inline bool_if_double_can_be_used<TAInt, TAFloat> equalsOpTmpl(TAFloat a, TAInt
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> greaterOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> greaterOp(A a, B b)
{
return greaterOpTmpl(a, b);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> greaterOp(A a, B b)
inline bool_if_safe_conversion<A, B> greaterOp(A a, B b)
{
return a > b;
}
@ -227,13 +227,13 @@ inline bool greaterOp<DB::UInt64, DB::Float32>(DB::UInt64 u, DB::Float32 f)
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> equalsOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> equalsOp(A a, B b)
{
return equalsOpTmpl(a, b);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> equalsOp(A a, B b)
inline bool_if_safe_conversion<A, B> equalsOp(A a, B b)
{
return a == b;
}
@ -288,52 +288,52 @@ inline bool equalsOp<DB::Int64, DB::Float32>(DB::Int64 u, DB::Float32 f)
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> notEqualsOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> notEqualsOp(A a, B b)
{
return !equalsOp(a, b);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> notEqualsOp(A a, B b)
inline bool_if_safe_conversion<A, B> notEqualsOp(A a, B b)
{
return a != b;
}
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> lessOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> lessOp(A a, B b)
{
return greaterOp(b, a);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> lessOp(A a, B b)
inline bool_if_safe_conversion<A, B> lessOp(A a, B b)
{
return a < b;
}
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> lessOrEqualsOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> lessOrEqualsOp(A a, B b)
{
return !greaterOp(a, b);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> lessOrEqualsOp(A a, B b)
inline bool_if_safe_conversion<A, B> lessOrEqualsOp(A a, B b)
{
return a <= b;
}
template <typename A, typename B>
inline bool_if_not_safe_convervsion<A, B> greaterOrEqualsOp(A a, B b)
inline bool_if_not_safe_conversion<A, B> greaterOrEqualsOp(A a, B b)
{
return !greaterOp(b, a);
}
template <typename A, typename B>
inline bool_if_safe_convervsion<A, B> greaterOrEqualsOp(A a, B b)
inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b)
{
return a >= b;
}

View File

@ -606,5 +606,13 @@ void Block::unshareColumns()
}
}
void Block::updateHash(SipHash & hash) const
{
for (size_t row_no = 0, num_rows = rows(); row_no < num_rows; ++row_no)
{
for (auto & col : getColumns())
col.column->updateHashWithValue(row_no, hash);
}
}
}

View File

@ -119,6 +119,12 @@ public:
*/
void unshareColumns();
/** Updates SipHash of the Block, using update method of columns.
* Returns hash for block, that could be used to differentiate blocks
* with same structure, but different data.
*/
void updateHash(SipHash & hash) const;
private:
void eraseImpl(size_t position);
void initializeIndexByName();

View File

@ -1,88 +1,88 @@
#pragma once
#define DBMS_NAME "ClickHouse"
#define DBMS_VERSION_MAJOR 1
#define DBMS_VERSION_MINOR 1
#define DBMS_NAME "ClickHouse"
#define DBMS_VERSION_MAJOR 1
#define DBMS_VERSION_MINOR 1
#define DBMS_DEFAULT_HOST "localhost"
#define DBMS_DEFAULT_PORT 9000
#define DBMS_DEFAULT_PORT_STR #DBMS_DEFAULT_PORT
#define DBMS_DEFAULT_HTTP_PORT 8123
#define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10
#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50
#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300
#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300
#define DBMS_DEFAULT_HOST "localhost"
#define DBMS_DEFAULT_PORT 9000
#define DBMS_DEFAULT_PORT_STR #DBMS_DEFAULT_PORT
#define DBMS_DEFAULT_HTTP_PORT 8123
#define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10
#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50
#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300
#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300
/// Timeout for synchronous request-result protocol call (like Ping or TablesStatus).
#define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5
#define DBMS_DEFAULT_POLL_INTERVAL 10
#define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5
#define DBMS_DEFAULT_POLL_INTERVAL 10
/// The size of the I/O buffer by default.
#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL
#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL
/// When writing data, a buffer of `max_compress_block_size` size is allocated for compression. When the buffer overflows or if into the buffer
/// more or equal data is written than `min_compress_block_size`, then with the next mark, the data will also compressed
/// As a result, for small columns (numbers 1-8 bytes), with index_granularity = 8192, the block size will be 64 KB.
/// And for large columns (Title - string ~100 bytes), the block size will be ~819 KB. Due to this, the compression ratio almost does not get worse.
#define DEFAULT_MIN_COMPRESS_BLOCK_SIZE 65536
#define DEFAULT_MAX_COMPRESS_BLOCK_SIZE 1048576
#define DEFAULT_MIN_COMPRESS_BLOCK_SIZE 65536
#define DEFAULT_MAX_COMPRESS_BLOCK_SIZE 1048576
/** Which blocks by default read the data (by number of rows).
* Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query.
*/
#define DEFAULT_BLOCK_SIZE 65536
#define DEFAULT_BLOCK_SIZE 65536
/** Which blocks should be formed for insertion into the table, if we control the formation of blocks.
* (Sometimes the blocks are inserted exactly such blocks that have been read / transmitted from the outside, and this parameter does not affect their size.)
* More than DEFAULT_BLOCK_SIZE, because in some tables a block of data on the disk is created for each block (quite a big thing),
* and if the parts were small, then it would be costly then to combine them.
*/
#define DEFAULT_INSERT_BLOCK_SIZE 1048576
#define DEFAULT_INSERT_BLOCK_SIZE 1048576
/** The same, but for merge operations. Less DEFAULT_BLOCK_SIZE for saving RAM (since all the columns are read).
* Significantly less, since there are 10-way mergers.
*/
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_MAX_QUERY_SIZE 262144
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
#define DEFAULT_MAX_QUERY_SIZE 262144
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
/// each period reduces the error counter by 2 times
/// too short a period can cause errors to disappear immediately after creation.
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC)
#define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Maximum waiting time in the request queue.
#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 16
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC)
#define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Maximum waiting time in the request queue.
#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 16
/// Used in the `reserve` method, when the number of rows is known, but their dimensions are unknown.
#define DBMS_APPROX_STRING_SIZE 64
/// Name suffix for the column containing the array offsets.
#define ARRAY_SIZES_COLUMN_NAME_SUFFIX ".size"
#define ARRAY_SIZES_COLUMN_NAME_SUFFIX ".size"
#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50264
#define DBMS_MIN_REVISION_WITH_TOTAL_ROWS_IN_PROGRESS 51554
#define DBMS_MIN_REVISION_WITH_BLOCK_INFO 51903
#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032
#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058
#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060
#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226
#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50264
#define DBMS_MIN_REVISION_WITH_TOTAL_ROWS_IN_PROGRESS 51554
#define DBMS_MIN_REVISION_WITH_BLOCK_INFO 51903
#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032
#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058
#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060
#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226
/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change.
#define DBMS_TCP_PROTOCOL_VERSION 54226
#define DBMS_TCP_PROTOCOL_VERSION 54226
#define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100
#define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100
/// The boundary on which the blocks for asynchronous file operations should be aligned.
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS 7500
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS 7500
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"
#if !defined(__x86_64__) && !defined(__aarch64__)
#error PLATFORM_NOT_SUPPORTED

View File

@ -177,7 +177,7 @@ namespace ErrorCodes
extern const int TOO_BIG_AST = 168;
extern const int BAD_TYPE_OF_FIELD = 169;
extern const int BAD_GET = 170;
extern const int BLOCKS_HAS_DIFFERENT_STRUCTURE = 171;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE = 171;
extern const int CANNOT_CREATE_DIRECTORY = 172;
extern const int CANNOT_ALLOCATE_MEMORY = 173;
extern const int CYCLIC_ALIASES = 174;

View File

@ -196,39 +196,4 @@ void FieldVisitorHash::operator() (const Array & x) const
applyVisitor(*this, elem);
}
UInt64 stringToDateOrDateTime(const String & s)
{
if (s.size() == strlen("YYYY-MM-DD"))
return stringToDate(s);
else
return stringToDateTime(s);
}
DayNum_t stringToDate(const String & s)
{
ReadBufferFromString in(s);
DayNum_t date{};
readDateText(date, in);
if (!in.eof())
throw Exception("String is too long for Date: " + s);
return date;
}
UInt64 stringToDateTime(const String & s)
{
ReadBufferFromString in(s);
time_t date_time{};
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + s);
return UInt64(date_time);
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Core/Field.h>
#include <Core/AccurateComparison.h>
#include <common/DateLUT.h>
@ -174,26 +175,12 @@ public:
};
/// Converts string with date or datetime (in format 'YYYY-MM-DD hh:mm:ss') to UInt64 containing numeric value of date (or datetime)
UInt64 stringToDateOrDateTime(const String & s);
/// Converts string with date to UInt16 (which is alias of DayNum_t) containing numeric value of date
DayNum_t stringToDate(const String & s);
/// Converts string with date to UInt64 containing numeric value of datetime
UInt64 stringToDateTime(const String & s);
/** More precise comparison, used for index.
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
* Except in cases when comparing signed and unsigned integers, which is unspecified behavior in FunctionsComparison,
* and when comparing integers and floats. Comparison is accurate here.
*/
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
using Double128 = long double; /// Non portable. Must have 64 bit mantissa to provide accurate comparisons.
public:
bool operator() (const Null & l, const Null & r) const { return true; }
bool operator() (const Null & l, const UInt64 & r) const { return false; }
@ -205,30 +192,30 @@ public:
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return r >= 0 && l == UInt64(r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return Double128(l) == Double128(r); }
bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return false; }
bool operator() (const UInt64 & l, const Array & r) const { return false; }
bool operator() (const UInt64 & l, const Tuple & r) const { return false; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l >= 0 && UInt64(l) == r; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Float64 & r) const { return Double128(l) == Double128(r); }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return false; }
bool operator() (const Int64 & l, const Array & r) const { return false; }
bool operator() (const Int64 & l, const Tuple & r) const { return false; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return Double128(l) == Double128(r); }
bool operator() (const Float64 & l, const Int64 & r) const { return Double128(l) == Double128(r); }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Float64 & l, const String & r) const { return false; }
bool operator() (const Float64 & l, const Array & r) const { return false; }
bool operator() (const Float64 & l, const Tuple & r) const { return false; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l == r; }
@ -254,8 +241,6 @@ public:
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
using Double128 = long double; /// Non portable. Must have 64 bit mantissa to provide accurate comparisons.
public:
bool operator() (const Null & l, const Null & r) const { return false; }
bool operator() (const Null & l, const UInt64 & r) const { return true; }
@ -267,30 +252,30 @@ public:
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return r >= 0 && l < UInt64(r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return Double128(l) < Double128(r); }
bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return true; }
bool operator() (const UInt64 & l, const Array & r) const { return true; }
bool operator() (const UInt64 & l, const Tuple & r) const { return true; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l < 0 || UInt64(l) < r; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Float64 & r) const { return Double128(l) < Double128(r); }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return true; }
bool operator() (const Int64 & l, const Array & r) const { return true; }
bool operator() (const Int64 & l, const Tuple & r) const { return true; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return Double128(l) < Double128(r); }
bool operator() (const Float64 & l, const Int64 & r) const { return Double128(l) < Double128(r); }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Float64 & l, const String & r) const { return true; }
bool operator() (const Float64 & l, const Array & r) const { return true; }
bool operator() (const Float64 & l, const Tuple & r) const { return true; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l < r; }

View File

@ -12,6 +12,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_NUMBER;
}
@ -33,7 +34,8 @@ static bool isParseError(int code)
|| code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|| code == ErrorCodes::CANNOT_PARSE_DATE
|| code == ErrorCodes::CANNOT_PARSE_DATETIME
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT;
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|| code == ErrorCodes::CANNOT_PARSE_NUMBER;
}

View File

@ -2,7 +2,7 @@
#include <Poco/Logger.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h> /// SubqueriesForSets
namespace Poco { class Logger; }

View File

@ -155,9 +155,11 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
else if (name == "PrettySpaceNoEscapes")
return std::make_shared<PrettySpaceBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows, context);
else if (name == "Vertical")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRowOutputStream>(buf, sample, context));
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRowOutputStream>(
buf, sample, settings.output_format_pretty_max_rows, context));
else if (name == "VerticalRaw")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRawRowOutputStream>(buf, sample, context));
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRawRowOutputStream>(
buf, sample, settings.output_format_pretty_max_rows, context));
else if (name == "Values")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<ValuesRowOutputStream>(buf));
else if (name == "JSON")

View File

@ -195,7 +195,7 @@ protected:
QuotaForIntervals * quota = nullptr; /// If nullptr - the quota is not used.
double prev_elapsed = 0;
/// The heirs must implement this function.
/// The successors must implement this function.
virtual Block readImpl() = 0;
/// Here you can do a preliminary initialization.

View File

@ -124,7 +124,7 @@ bool JSONEachRowRowInputStream::read(Block & block)
}
skipWhitespaceIfAny(istr);
if (!istr.eof() && *istr.position() == ',')
if (!istr.eof() && (*istr.position() == ',' || *istr.position() == ';')) /// Semicolon is added for convenience as it could be used at end of INSERT query.
++istr.position();
/// Fill non-visited columns with the default values.

View File

@ -11,7 +11,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
extern const int BLOCKS_HAS_DIFFERENT_STRUCTURE;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE;
}
@ -130,7 +130,7 @@ void MergingSortedBlockInputStream::init(Block & merged_block, ColumnPlainPtrs &
{
throw Exception("Merging blocks has different names or types of columns:\n"
+ shared_block_ptr->dumpStructure() + "\nand\n" + merged_block.dumpStructure(),
ErrorCodes::BLOCKS_HAS_DIFFERENT_STRUCTURE);
ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
}
}
}

View File

@ -8,8 +8,8 @@ namespace DB
{
/** Provides reading from a Buffer, taking exclusive ownership over it's lifetime,
* simplifies usage of ReadBufferFromFile (no need to manage buffer lifetime) etc.
*/
* simplifies usage of ReadBufferFromFile (no need to manage buffer lifetime) etc.
*/
template <typename OwnType>
class OwningBlockInputStream : public IProfilingBlockInputStream
{

View File

@ -18,7 +18,7 @@ namespace DB
class PushingToViewsBlockOutputStream : public IBlockOutputStream
{
public:
PushingToViewsBlockOutputStream(String database, String table, const Context & context_, ASTPtr query_ptr_)
PushingToViewsBlockOutputStream(String database, String table, const Context & context_, const ASTPtr & query_ptr_)
: context(context_), query_ptr(query_ptr_)
{
storage = context.getTable(database, table);

View File

@ -102,7 +102,7 @@ void RemoteBlockInputStream::sendExternalTables()
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(std::make_shared<OneBlockInputStream>(cur->getSampleBlock()), table.first));
@ -122,7 +122,7 @@ Block RemoteBlockInputStream::readImpl()
{
sendQuery();
if (settings.skip_unavailable_shards && (0 == multiplexed_connections->size()))
if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size()))
return {};
}
@ -224,7 +224,7 @@ void RemoteBlockInputStream::readSuffixImpl()
void RemoteBlockInputStream::createMultiplexedConnections()
{
Settings * multiplexed_connections_settings = send_settings ? &settings : nullptr;
Settings * multiplexed_connections_settings = send_settings ? &context.getSettingsRef() : nullptr;
const QualifiedTableName * main_table_ptr = main_table ? &main_table.value() : nullptr;
if (connection != nullptr)
multiplexed_connections = std::make_unique<MultiplexedConnections>(
@ -241,12 +241,12 @@ void RemoteBlockInputStream::createMultiplexedConnections()
throw Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
void RemoteBlockInputStream::init(const Settings * settings_)
void RemoteBlockInputStream::init(const Settings * settings)
{
if (settings_)
if (settings)
{
send_settings = true;
settings = *settings_;
context.setSettings(*settings);
}
else
send_settings = false;
@ -256,7 +256,7 @@ void RemoteBlockInputStream::sendQuery()
{
createMultiplexedConnections();
if (settings.skip_unavailable_shards && 0 == multiplexed_connections->size())
if (context.getSettingsRef().skip_unavailable_shards && 0 == multiplexed_connections->size())
return;
established = true;

View File

@ -90,7 +90,7 @@ protected:
bool hasThrownException() const;
private:
void init(const Settings * settings_);
void init(const Settings * settings);
void sendQuery();
@ -118,7 +118,6 @@ private:
const String query;
bool send_settings;
Settings settings;
/// If != nullptr, used to limit network trafic
ThrottlerPtr throttler;
/// Temporary tables needed to be sent to remote servers

View File

@ -10,6 +10,7 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
extern const int CANNOT_READ_ALL_DATA;
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
}
@ -108,6 +109,7 @@ bool TSKVRowInputStream::read(Block & block)
{
StringRef name_ref;
bool has_value = readName(istr, name_ref, name_buf);
ssize_t index = -1;
if (has_value)
{
@ -126,7 +128,7 @@ bool TSKVRowInputStream::read(Block & block)
}
else
{
size_t index = it->second;
index = it->second;
if (read_columns[index])
throw Exception("Duplicate field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
@ -159,7 +161,16 @@ bool TSKVRowInputStream::read(Block & block)
break;
}
else
throw Exception("Found garbage after field in TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
{
/// Possibly a garbage was written into column, remove it
if (index >= 0)
{
block.getByPosition(index).column->popBack(1);
read_columns[index] = false;
}
throw Exception("Found garbage after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
}
}
}

View File

@ -10,8 +10,9 @@
namespace DB
{
VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context)
: ostr(ostr_), sample(sample_)
VerticalRowOutputStream::VerticalRowOutputStream(
WriteBuffer & ostr_, const Block & sample_, size_t max_rows_, const Context & context)
: ostr(ostr_), sample(sample_), max_rows(max_rows_)
{
size_t columns = sample.columns();
@ -60,6 +61,9 @@ void VerticalRowOutputStream::flush()
void VerticalRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
if (row_number > max_rows)
return;
writeString(names_and_paddings[field_number], ostr);
writeValue(column, type, row_num);
writeChar('\n', ostr);
@ -82,6 +86,10 @@ void VerticalRawRowOutputStream::writeValue(const IColumn & column, const IDataT
void VerticalRowOutputStream::writeRowStartDelimiter()
{
++row_number;
if (row_number > max_rows)
return;
writeCString("Row ", ostr);
writeIntText(row_number, ostr);
writeCString(":\n", ostr);
@ -95,9 +103,77 @@ void VerticalRowOutputStream::writeRowStartDelimiter()
void VerticalRowOutputStream::writeRowBetweenDelimiter()
{
if (row_number > max_rows)
return;
writeCString("\n", ostr);
field_number = 0;
}
void VerticalRowOutputStream::writeSuffix()
{
if (row_number > max_rows)
{
writeCString("Showed first ", ostr);
writeIntText(max_rows, ostr);
writeCString(".\n", ostr);
}
if (totals || extremes)
{
writeCString("\n", ostr);
writeTotals();
writeExtremes();
}
}
void VerticalRowOutputStream::writeSpecialRow(const Block & block, size_t row_num, const char * title)
{
writeCString("\n", ostr);
row_number = 0;
field_number = 0;
size_t columns = block.columns();
writeCString(title, ostr);
writeCString(":\n", ostr);
size_t width = strlen(title) + 1;
for (size_t i = 0; i < width; ++i)
writeCString("", ostr);
writeChar('\n', ostr);
for (size_t i = 0; i < columns; ++i)
{
if (i != 0)
writeFieldDelimiter();
auto & col = block.getByPosition(i);
writeField(*col.column.get(), *col.type.get(), row_num);
}
}
void VerticalRowOutputStream::writeTotals()
{
if (totals)
{
writeSpecialRow(totals, 0, "Totals");
}
}
void VerticalRowOutputStream::writeExtremes()
{
if (extremes)
{
writeSpecialRow(extremes, 0, "Min");
writeSpecialRow(extremes, 1, "Max");
}
}
}

View File

@ -18,24 +18,37 @@ class Context;
class VerticalRowOutputStream : public IRowOutputStream
{
public:
VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context);
VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_, size_t max_rows_, const Context & context);
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeRowStartDelimiter() override;
void writeRowBetweenDelimiter() override;
void writeSuffix() override;
void flush() override;
void setTotals(const Block & totals_) override { totals = totals_; }
void setExtremes(const Block & extremes_) override { extremes = extremes_; }
protected:
virtual void writeValue(const IColumn & column, const IDataType & type, size_t row_num) const;
void writeTotals();
void writeExtremes();
/// For totals and extremes.
void writeSpecialRow(const Block & block, size_t row_num, const char * title);
WriteBuffer & ostr;
const Block sample;
size_t max_rows;
size_t field_number = 0;
size_t row_number = 0;
using NamesAndPaddings = std::vector<String>;
NamesAndPaddings names_and_paddings;
Block totals;
Block extremes;
};
@ -44,8 +57,7 @@ protected:
class VerticalRawRowOutputStream final : public VerticalRowOutputStream
{
public:
VerticalRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context)
: VerticalRowOutputStream(ostr_, sample_, context) {}
using VerticalRowOutputStream::VerticalRowOutputStream;
protected:
void writeValue(const IColumn & column, const IDataType & type, size_t row_num) const override;

View File

@ -50,7 +50,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in;
in = table->read(column_names, 0, context, Settings(), stage)[0];
in = table->read(column_names, 0, context, stage)[0];
in = std::make_shared<ExpressionBlockInputStream>(in, expression);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));

View File

@ -55,7 +55,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
BlockInputStreamPtr in = table->read(column_names, 0, context, stage)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 1);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));

View File

@ -126,7 +126,7 @@ int main(int argc, char ** argv)
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
BlockInputStreamPtr in = table->read(column_names, 0, context, stage)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 4);
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -72,7 +72,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
BlockInputStreamPtr in = table->read(column_names, 0, context, stage)[0];
ForkBlockInputStreams fork(in);

View File

@ -5,6 +5,7 @@
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Interpreters/Context.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/executeQuery.h>

View File

@ -101,7 +101,7 @@ try
if (argc == 2 && 0 == strcmp(argv[1], "read"))
{
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, Context{}, Settings(), stage)[0];
BlockInputStreamPtr in = table->read(column_names, 0, Context{}, stage)[0];
WriteBufferFromFileDescriptor out1(STDOUT_FILENO);
CompressedWriteBuffer out2(out1);
NativeBlockOutputStream out3(out2, ClickHouseRevision::get());

View File

@ -143,7 +143,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0];
BlockInputStreamPtr in = table->read(column_names, 0, Context{}, stage, argc == 2 ? atoi(argv[1]) : 1048576)[0];
in = std::make_shared<PartialSortingBlockInputStream>(in, sort_columns);
in = std::make_shared<MergeSortingBlockInputStream>(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -35,9 +35,9 @@ void test1()
QueryProcessingStage::Enum stage3;
BlockInputStreams streams;
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage1, 1)[0], 30, 30000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage2, 1)[0], 30, 2000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage3, 1)[0], 30, 100));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage1, 1)[0], 30, 30000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage2, 1)[0], 30, 2000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage3, 1)[0], 30, 100));
UnionBlockInputStream<> union_stream(streams, nullptr, 2);
@ -85,15 +85,15 @@ void test2()
BlockInputStreams streams;
BlockInputStreamPtr stream1 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage1, 1)[0], 30, 30000);
BlockInputStreamPtr stream1 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage1, 1)[0], 30, 30000);
stream1 = std::make_shared<BlockExtraInfoInputStream>(stream1, extra_info1);
streams.emplace_back(stream1);
BlockInputStreamPtr stream2 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage2, 1)[0], 30, 2000);
BlockInputStreamPtr stream2 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage2, 1)[0], 30, 2000);
stream2 = std::make_shared<BlockExtraInfoInputStream>(stream2, extra_info2);
streams.emplace_back(stream2);
BlockInputStreamPtr stream3 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, Settings(), stage3, 1)[0], 30, 100);
BlockInputStreamPtr stream3 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage3, 1)[0], 30, 100);
stream3 = std::make_shared<BlockExtraInfoInputStream>(stream3, extra_info3);
streams.emplace_back(stream3);

View File

@ -35,7 +35,7 @@ try
StoragePtr table = context.getTable("default", "hits6");
QueryProcessingStage::Enum stage;
BlockInputStreams streams = table->read(column_names, nullptr, context, settings, stage, settings.max_block_size, settings.max_threads);
BlockInputStreams streams = table->read(column_names, nullptr, context, stage, settings.max_block_size, settings.max_threads);
for (size_t i = 0, size = streams.size(); i < size; ++i)
streams[i] = std::make_shared<AsynchronousBlockInputStream>(streams[i]);

View File

@ -7,6 +7,7 @@
#include <Common/escapeForFileName.h>
#include <Common/StringUtils.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
@ -162,10 +163,6 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
}
};
/** `packaged_task` is used so that exceptions are automatically passed to the main thread.
* Disadvantage - exceptions fall into the main thread only after the end of all tasks.
*/
const size_t bunch_size = TABLES_PARALLEL_LOAD_BUNCH_SIZE;
size_t num_bunches = (total_tables + bunch_size - 1) / bunch_size;

View File

@ -1,7 +1,10 @@
#include <sstream>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Storages/StorageFactory.h>
#include <Databases/DatabasesCommon.h>

View File

@ -5,11 +5,14 @@
#include <Storages/IStorage.h>
#include <Databases/IDatabase.h>
/// General functionality for several different database engines.
namespace DB
{
class Context;
/** Get the row with the table definition based on the CREATE query.
* It is an ATTACH query that you can execute to create a table from the correspondent database.

View File

@ -402,10 +402,14 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
[] (auto & pair) { return pair.first; });
/// request new values
update(required_ids, [&] (const auto id, const auto) {
update(required_ids,
[&] (const auto id, const auto)
{
for (const auto row : outdated_ids[id])
out[row] = true;
}, [&] (const auto id, const auto) {
},
[&] (const auto id, const auto)
{
for (const auto row : outdated_ids[id])
out[row] = false;
});
@ -594,18 +598,19 @@ void CacheDictionary::getItemsNumberImpl(
[] (auto & pair) { return pair.first; });
/// request new values
update(required_ids, [&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
for (const auto row : outdated_ids[id])
out[row] = attribute_value;
},
[&] (const auto id, const auto cell_idx)
{
for (const auto row : outdated_ids[id])
out[row] = get_default(row);
});
for (const auto row : outdated_ids[id])
out[row] = attribute_value;
},
[&] (const auto id, const auto cell_idx)
{
for (const auto row : outdated_ids[id])
out[row] = get_default(row);
});
}
template <typename DefaultGetter>
@ -715,12 +720,16 @@ void CacheDictionary::getItemsString(
std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids),
[] (auto & pair) { return pair.first; });
update(required_ids, [&] (const auto id, const auto cell_idx) {
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id].size();
}, [&] (const auto id, const auto cell_idx) {
},
[&] (const auto id, const auto cell_idx)
{
for (const auto row : outdated_ids[id])
total_length += get_default(row).size + 1;
});
@ -740,7 +749,8 @@ void CacheDictionary::getItemsString(
template <typename PresentIdHandler, typename AbsentIdHandler>
void CacheDictionary::update(
const std::vector<Key> & requested_ids, PresentIdHandler && on_cell_updated,
const std::vector<Key> & requested_ids,
PresentIdHandler && on_cell_updated,
AbsentIdHandler && on_id_not_found) const
{
std::unordered_map<Key, UInt8> remaining_ids{requested_ids.size()};
@ -773,7 +783,8 @@ void CacheDictionary::update(
const auto & ids = id_column->getData();
/// cache column pointers
const auto column_ptrs = ext::map<std::vector>(ext::range(0, attributes.size()), [&block] (const auto & i) {
const auto column_ptrs = ext::map<std::vector>(ext::range(0, attributes.size()), [&block] (size_t i)
{
return block.safeGetByPosition(i + 1).column.get();
});

View File

@ -83,7 +83,7 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadIds(const std::vector<UInt64
BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
return createStreamForSelectiveLoad(
query_builder.composeLoadKeysQuery(

View File

@ -31,7 +31,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override { return true; }
bool supportsSelectiveLoad() const override { return true; }

View File

@ -69,7 +69,7 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const ComplexKeyCacheDictio
#define DECLARE(TYPE)\
void ComplexKeyCacheDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -97,7 +97,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyCacheDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
dict_struct.validateKeyTypes(key_types);
@ -115,7 +115,7 @@ void ComplexKeyCacheDictionary::getString(
#define DECLARE(TYPE)\
void ComplexKeyCacheDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -141,7 +141,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyCacheDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
dict_struct.validateKeyTypes(key_types);
@ -157,7 +157,7 @@ void ComplexKeyCacheDictionary::getString(
#define DECLARE(TYPE)\
void ComplexKeyCacheDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -183,7 +183,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyCacheDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const
{
dict_struct.validateKeyTypes(key_types);
@ -243,7 +243,7 @@ ComplexKeyCacheDictionary::FindResult ComplexKeyCacheDictionary::findCellIdx(con
}
void ComplexKeyCacheDictionary::has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
{
dict_struct.validateKeyTypes(key_types);
@ -409,7 +409,7 @@ ComplexKeyCacheDictionary::Attribute ComplexKeyCacheDictionary::createAttributeW
template <typename OutputType, typename DefaultGetter>
void ComplexKeyCacheDictionary::getItemsNumber(
Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultGetter && get_default) const
{
@ -435,7 +435,7 @@ void ComplexKeyCacheDictionary::getItemsNumber(
template <typename AttributeType, typename OutputType, typename DefaultGetter>
void ComplexKeyCacheDictionary::getItemsNumberImpl(
Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultGetter && get_default) const
{
@ -513,7 +513,7 @@ void ComplexKeyCacheDictionary::getItemsNumberImpl(
template <typename DefaultGetter>
void ComplexKeyCacheDictionary::getItemsString(
Attribute & attribute, const ConstColumnPlainPtrs & key_columns, ColumnString * out,
Attribute & attribute, const Columns & key_columns, ColumnString * out,
DefaultGetter && get_default) const
{
const auto rows_num = key_columns.front()->size();
@ -654,18 +654,16 @@ void ComplexKeyCacheDictionary::getItemsString(
template <typename PresentKeyHandler, typename AbsentKeyHandler>
void ComplexKeyCacheDictionary::update(
const ConstColumnPlainPtrs & in_key_columns, const PODArray<StringRef> & in_keys,
const std::vector<size_t> & in_requested_rows, PresentKeyHandler && on_cell_updated,
const Columns & in_key_columns, const PODArray<StringRef> & in_keys,
const std::vector<size_t> & in_requested_rows,
PresentKeyHandler && on_cell_updated,
AbsentKeyHandler && on_key_not_found) const
{
MapType<bool> remaining_keys{in_requested_rows.size()};
for (const auto row : in_requested_rows)
remaining_keys.insert({ in_keys[row], false });
std::uniform_int_distribution<UInt64> distribution{
dict_lifetime.min_sec,
dict_lifetime.max_sec
};
std::uniform_int_distribution<UInt64> distribution(dict_lifetime.min_sec, dict_lifetime.max_sec);
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
{
@ -682,18 +680,18 @@ void ComplexKeyCacheDictionary::update(
while (const auto block = stream->read())
{
/// cache column pointers
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
const auto key_columns = ext::map<Columns>(
ext::range(0, keys_size),
[&] (const size_t attribute_idx)
{
return block.safeGetByPosition(attribute_idx).column.get();
return block.safeGetByPosition(attribute_idx).column;
});
const auto attribute_columns = ext::map<ConstColumnPlainPtrs>(
const auto attribute_columns = ext::map<Columns>(
ext::range(0, attributes_size),
[&] (const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column.get();
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
const auto rows_num = block.rows();
@ -896,7 +894,7 @@ ComplexKeyCacheDictionary::Attribute & ComplexKeyCacheDictionary::getAttribute(c
return attributes[it->second];
}
StringRef ComplexKeyCacheDictionary::allocKey(const size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys) const
StringRef ComplexKeyCacheDictionary::allocKey(const size_t row, const Columns & key_columns, StringRefs & keys) const
{
if (key_size_is_fixed)
return placeKeysInFixedSizePool(row, key_columns);
@ -914,7 +912,7 @@ void ComplexKeyCacheDictionary::freeKey(const StringRef key) const
template <typename Arena>
StringRef ComplexKeyCacheDictionary::placeKeysInPool(
const size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
@ -937,7 +935,7 @@ StringRef ComplexKeyCacheDictionary::placeKeysInPool(
}
StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool(
const size_t row, const ConstColumnPlainPtrs & key_columns) const
const size_t row, const Columns & key_columns) const
{
const auto res = fixed_size_keys_pool->alloc();
auto place = res;

View File

@ -87,7 +87,7 @@ public:
/// See the requirement in IDataType.h for text-serialization functions.
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -102,12 +102,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -122,12 +122,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -142,10 +142,10 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const;
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
private:
template <typename Value> using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
@ -196,26 +196,27 @@ private:
template <typename OutputType, typename DefaultGetter>
void getItemsNumber(
Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename DefaultGetter>
void getItemsNumberImpl(
Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultGetter && get_default) const;
template <typename DefaultGetter>
void getItemsString(
Attribute & attribute, const ConstColumnPlainPtrs & key_columns, ColumnString * out,
Attribute & attribute, const Columns & key_columns, ColumnString * out,
DefaultGetter && get_default) const;
template <typename PresentKeyHandler, typename AbsentKeyHandler>
void update(
const ConstColumnPlainPtrs & in_key_columns, const PODArray<StringRef> & in_keys,
const std::vector<std::size_t> & in_requested_rows, PresentKeyHandler && on_cell_updated,
const Columns & in_key_columns, const PODArray<StringRef> & in_keys,
const std::vector<std::size_t> & in_requested_rows,
PresentKeyHandler && on_cell_updated,
AbsentKeyHandler && on_key_not_found) const;
UInt64 getCellIdx(const StringRef key) const;
@ -226,16 +227,16 @@ private:
Attribute & getAttribute(const std::string & attribute_name) const;
StringRef allocKey(const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys) const;
StringRef allocKey(const std::size_t row, const Columns & key_columns, StringRefs & keys) const;
void freeKey(const StringRef key) const;
template <typename Arena>
static StringRef placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool);
const std::size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool);
StringRef placeKeysInFixedSizePool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns) const;
const std::size_t row, const Columns & key_columns) const;
static StringRef copyIntoArena(StringRef src, Arena & arena);
StringRef copyKey(const StringRef key) const;

View File

@ -42,7 +42,7 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(const ComplexKeyHashedDic
#define DECLARE(TYPE)\
void ComplexKeyHashedDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -72,7 +72,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyHashedDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
dict_struct.validateKeyTypes(key_types);
@ -92,7 +92,7 @@ void ComplexKeyHashedDictionary::getString(
#define DECLARE(TYPE)\
void ComplexKeyHashedDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -120,7 +120,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyHashedDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
dict_struct.validateKeyTypes(key_types);
@ -138,7 +138,7 @@ void ComplexKeyHashedDictionary::getString(
#define DECLARE(TYPE)\
void ComplexKeyHashedDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const\
{\
dict_struct.validateKeyTypes(key_types);\
@ -166,7 +166,7 @@ DECLARE(Float64)
#undef DECLARE
void ComplexKeyHashedDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const
{
dict_struct.validateKeyTypes(key_types);
@ -182,7 +182,7 @@ void ComplexKeyHashedDictionary::getString(
[&] (const std::size_t) { return StringRef{def}; });
}
void ComplexKeyHashedDictionary::has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
void ComplexKeyHashedDictionary::has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
{
dict_struct.validateKeyTypes(key_types);
@ -237,14 +237,14 @@ void ComplexKeyHashedDictionary::loadData()
const auto rows = block.rows();
element_count += rows;
const auto key_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, keys_size),
const auto key_column_ptrs = ext::map<Columns>(ext::range(0, keys_size),
[&] (const std::size_t attribute_idx) {
return block.safeGetByPosition(attribute_idx).column.get();
return block.safeGetByPosition(attribute_idx).column;
});
const auto attribute_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, attributes_size),
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size),
[&] (const std::size_t attribute_idx) {
return block.safeGetByPosition(keys_size + attribute_idx).column.get();
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
for (const auto row_idx : ext::range(0, rows))
@ -356,7 +356,7 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void ComplexKeyHashedDictionary::getItemsNumber(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const
{
@ -382,7 +382,7 @@ void ComplexKeyHashedDictionary::getItemsNumber(
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void ComplexKeyHashedDictionary::getItemsImpl(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const
{
@ -456,7 +456,7 @@ const ComplexKeyHashedDictionary::Attribute & ComplexKeyHashedDictionary::getAtt
}
StringRef ComplexKeyHashedDictionary::placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
const std::size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
@ -479,7 +479,7 @@ StringRef ComplexKeyHashedDictionary::placeKeysInPool(
}
template <typename T>
void ComplexKeyHashedDictionary::has(const Attribute & attribute, const ConstColumnPlainPtrs & key_columns, PaddedPODArray<UInt8> & out) const
void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const
{
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto keys_size = key_columns.size();

View File

@ -65,7 +65,7 @@ public:
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -80,12 +80,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -100,12 +100,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -120,10 +120,10 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const;
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
private:
template <typename Value> using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
@ -163,14 +163,14 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsNumber(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsImpl(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const;
@ -183,10 +183,10 @@ private:
const Attribute & getAttribute(const std::string & attribute_name) const;
static StringRef placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool);
const std::size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool);
template <typename T>
void has(const Attribute & attribute, const ConstColumnPlainPtrs & key_columns, PaddedPODArray<UInt8> & out) const;
void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const;
const std::string name;
const DictionaryStructure dict_struct;

View File

@ -0,0 +1,50 @@
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Block.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataStreams/IBlockOutputStream.h>
namespace DB
{
/// For simple key
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids)
{
auto column = std::make_shared<ColumnUInt64>(ids.size());
memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front()));
Block block{{ std::move(column), std::make_shared<DataTypeUInt64>(), "id" }};
out->writePrefix();
out->write(block);
out->writeSuffix();
out->flush();
}
/// For composite key
void formatKeys(const DictionaryStructure & dict_struct, BlockOutputStreamPtr & out,
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
Block block;
for (size_t i = 0, size = key_columns.size(); i < size; ++i)
{
const ColumnPtr & source_column = key_columns[i];
ColumnPtr filtered_column = source_column->cloneEmpty();
filtered_column->reserve(requested_rows.size());
for (size_t idx : requested_rows)
filtered_column->insertFrom(*source_column, idx);
block.insert({ filtered_column, (*dict_struct.key)[i].type, toString(i) });
}
out->writePrefix();
out->write(block);
out->writeSuffix();
out->flush();
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <vector>
#include <common/Types.h>
#include <Columns/IColumn.h>
namespace DB
{
class IBlockOutputStream;
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
struct DictionaryStructure;
/// Write keys to block output stream.
/// For simple key
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids);
/// For composite key
void formatKeys(const DictionaryStructure & dict_struct, BlockOutputStreamPtr & out,
const Columns & key_columns, const std::vector<std::size_t> & requested_rows);
}

View File

@ -1,8 +1,12 @@
#include <thread>
#include <future>
#include <Dictionaries/ExecutableDictionarySource.h>
#include <Common/ShellCommand.h>
#include <Interpreters/Context.h>
#include <Dictionaries/OwningBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataTypes/DataTypesNumber.h>
@ -42,77 +46,92 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll " + toString());
auto process = ShellCommand::execute(command);
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ShellCommand>>(input_stream, std::move(process));
}
void idsToBuffer(const Context & context, const std::string & format, Block & sample_block, WriteBuffer & out_buffer,
const std::vector<UInt64> & ids)
{
ColumnWithTypeAndName column;
column.type = std::make_shared<DataTypeUInt64>();
column.column = column.type->createColumn();
for (auto & id : ids)
/** A stream, that also runs and waits for background thread
* (that will feed data into pipe to be read from the other side of the pipe).
*/
class BlockInputStreamWithBackgroundThread : public IProfilingBlockInputStream
{
public:
BlockInputStreamWithBackgroundThread(
const BlockInputStreamPtr & stream_, std::unique_ptr<ShellCommand> && command_,
std::packaged_task<void()> && task_)
: stream{stream_}, command{std::move(command_)}, task(std::move(task_)),
thread([this]{ task(); command->in.close(); })
{
column.column->insert(id); //CHECKME maybe faster?
children.push_back(stream);
}
Block block;
block.insert(std::move(column));
auto stream_out = context.getOutputFormat(format, out_buffer, sample_block);
stream_out->writePrefix();
stream_out->write(block);
stream_out->writeSuffix();
stream_out->flush();
}
void columnsToBuffer(const Context & context, const std::string & format, Block & sample_block, WriteBuffer & out_buffer, const DictionaryStructure & dict_struct,
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
{
Block block;
const auto keys_size = key_columns.size();
for (const auto i : ext::range(0, keys_size))
~BlockInputStreamWithBackgroundThread() override
{
const auto & key_description = (*dict_struct.key)[i];
const auto & key = key_columns[i];
ColumnWithTypeAndName column;
column.type = key_description.type;
column.column = key->clone(); // CHECKME !!
block.insert(std::move(column));
if (thread.joinable())
{
try
{
readSuffix();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
auto stream_out = context.getOutputFormat(format, out_buffer, sample_block);
stream_out->writePrefix();
stream_out->write(block);
stream_out->writeSuffix();
stream_out->flush();
}
private:
Block readImpl() override { return stream->read(); }
void readSuffix() override
{
thread.join();
/// To rethrow an exception, if any.
task.get_future().get();
}
String getName() const override { return "WithBackgroundThread"; }
String getID() const override { return "WithBackgroundThread(" + stream->getID() + ")"; }
BlockInputStreamPtr stream;
std::unique_ptr<ShellCommand> command;
std::packaged_task<void()> task;
std::thread thread;
};
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds " + toString() + " ids=" + std::to_string(ids.size()));
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
auto process = ShellCommand::execute(command);
idsToBuffer(context, format, sample_block, process->in, ids);
process->in.close();
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<BlockInputStreamWithBackgroundThread>(
input_stream, std::move(process), std::packaged_task<void()>(
[output_stream, &ids, this]() mutable
{
formatIDs(output_stream, ids);
}));
}
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys " + toString() + " rows=" + std::to_string(requested_rows.size()));
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
auto process = ShellCommand::execute(command);
columnsToBuffer(context, format, sample_block, process->in, dict_struct, key_columns, requested_rows);
process->in.close();
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
return std::make_shared<BlockInputStreamWithBackgroundThread>(
input_stream, std::move(process), std::packaged_task<void()>(
[output_stream, key_columns, &requested_rows, this]() mutable
{
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
}));
}
bool ExecutableDictionarySource::isModified() const

View File

@ -28,7 +28,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
bool isModified() const override;
@ -48,9 +48,4 @@ private:
const Context & context;
};
void idsToBuffer(const Context & context, const std::string & format, Block & sample_block, WriteBuffer & out_stream,
const std::vector<UInt64> & ids);
void columnsToBuffer(const Context & context, const std::string & format, Block & sample_block, WriteBuffer & out_buffer,
const DictionaryStructure & dict_struct, const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows);
}

View File

@ -211,7 +211,7 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector<UInt64>
std::string ExternalQueryBuilder::composeLoadKeysQuery(
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
const std::vector<std::size_t> & requested_rows,
LoadKeysMethod method)
{
@ -300,7 +300,7 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery(
}
void ExternalQueryBuilder::composeKeyCondition(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
void ExternalQueryBuilder::composeKeyCondition(const Columns & key_columns, const std::size_t row, WriteBuffer & out) const
{
writeString("(", out);
@ -348,7 +348,7 @@ std::string ExternalQueryBuilder::composeKeyTupleDefinition() const
}
void ExternalQueryBuilder::composeKeyTuple(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
void ExternalQueryBuilder::composeKeyTuple(const Columns & key_columns, const std::size_t row, WriteBuffer & out) const
{
writeString("(", out);

View File

@ -57,20 +57,20 @@ struct ExternalQueryBuilder
};
std::string composeLoadKeysQuery(
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
const std::vector<size_t> & requested_rows,
LoadKeysMethod method);
private:
/// Expression in form (x = c1 AND y = c2 ...)
void composeKeyCondition(const ConstColumnPlainPtrs & key_columns, const size_t row, WriteBuffer & out) const;
void composeKeyCondition(const Columns & key_columns, const size_t row, WriteBuffer & out) const;
/// Expression in form (x, y, ...)
std::string composeKeyTupleDefinition() const;
/// Expression in form (c1, c2, ...)
void composeKeyTuple(const ConstColumnPlainPtrs & key_columns, const size_t row, WriteBuffer & out) const;
void composeKeyTuple(const Columns & key_columns, const size_t row, WriteBuffer & out) const;
/// Write string with specified quoting style.
void writeQuoted(const std::string & s, WriteBuffer & out) const;

View File

@ -60,7 +60,7 @@ void ExternalResultDescription::init(const Block & sample_block_)
ErrorCodes::UNKNOWN_TYPE};
names.emplace_back(column.name);
sample_columns.emplace_back(column.column.get());
sample_columns.emplace_back(column.column);
}
}

View File

@ -30,7 +30,7 @@ struct ExternalResultDescription
Block sample_block;
std::vector<ValueType> types;
std::vector<std::string> names;
ConstColumnPlainPtrs sample_columns;
Columns sample_columns;
void init(const Block & sample_block_);
};

View File

@ -1,5 +1,5 @@
#include <Interpreters/Context.h>
#include <Dictionaries/OwningBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <Dictionaries/FileDictionarySource.h>
#include <IO/ReadBufferFromFile.h>
#include <Poco/File.h>

View File

@ -27,7 +27,7 @@ public:
}
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override
{
throw Exception{"Method loadKeys is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
}

View File

@ -2,12 +2,12 @@
#include <Poco/Net/HTTPRequest.h>
#include <Interpreters/Context.h>
#include <Dictionaries/OwningBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteBufferFromOStream.h>
#include <Dictionaries/ExecutableDictionarySource.h> // idsToBuffer, columnsToBuffer
#include <Dictionaries/DictionarySourceHelpers.h>
#include <common/logger_useful.h>
@ -44,40 +44,43 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll()
LOG_TRACE(log, "loadAll " + toString());
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_GET);
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(stream, std::move(in_ptr));
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds " + toString() + " ids=" + std::to_string(ids.size()));
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & out_stream) {
WriteBufferFromOStream out_buffer(out_stream);
idsToBuffer(context, format, sample_block, out_buffer, ids);
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
{
WriteBufferFromOStream out_buffer(ostr);
auto output_stream = context.getOutputFormat(format, out_buffer, sample_block);
formatIDs(output_stream, ids);
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback);
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(stream, std::move(in_ptr));
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
BlockInputStreamPtr HTTPDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys " + toString() + " rows=" + std::to_string(requested_rows.size()));
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & out_stream) {
WriteBufferFromOStream out_buffer(out_stream);
columnsToBuffer(context, format, sample_block, out_buffer, dict_struct, key_columns, requested_rows);
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
{
WriteBufferFromOStream out_buffer(ostr);
auto output_stream = context.getOutputFormat(format, out_buffer, sample_block);
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback);
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(stream, std::move(in_ptr));
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
bool HTTPDictionarySource::isModified() const

View File

@ -27,7 +27,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
bool isModified() const override;

View File

@ -16,21 +16,25 @@ using DictionarySourcePtr = std::unique_ptr<IDictionarySource>;
class IDictionarySource
{
public:
/// returns an input stream with all the data available from this source
/// Returns an input stream with all the data available from this source.
virtual BlockInputStreamPtr loadAll() = 0;
/** Indicates whether this source supports "random access" loading of data
* loadId and loadIds can only be used if this function returns true.
*/
* loadId and loadIds can only be used if this function returns true.
*/
virtual bool supportsSelectiveLoad() const = 0;
/// returns an input stream with the data for a collection of identifiers
/** Returns an input stream with the data for a collection of identifiers.
* It must be guaranteed, that 'ids' array will live at least until all data will be read from returned stream.
*/
virtual BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) = 0;
/** returns an input stream with the data for a collection of composite keys.
* `requested_rows` contains indices of all rows containing unique keys. */
/** Returns an input stream with the data for a collection of composite keys.
* `requested_rows` contains indices of all rows containing unique keys.
* It must be guaranteed, that 'requested_rows' array will live at least until all data will be read from returned stream.
*/
virtual BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) = 0;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) = 0;
/// indicates whether the source has been modified since last load* operation
virtual bool isModified() const = 0;

View File

@ -149,7 +149,7 @@ Block MongoDBBlockInputStream::readImpl()
const size_t size = columns.size();
for (const auto i : ext::range(0, size))
columns[i] = block.safeGetByPosition(i).column.get();
columns[i] = block.getByPosition(i).column.get();
size_t num_rows = 0;
while (num_rows < max_block_size)

View File

@ -221,7 +221,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> &
BlockInputStreamPtr MongoDBDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
if (!dict_struct.key)
throw Exception{"'key' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};

View File

@ -47,7 +47,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field
bool isModified() const override { return true; }

View File

@ -76,7 +76,7 @@ Block MySQLBlockInputStream::readImpl()
/// cache pointers returned by the calls to getByPosition
std::vector<IColumn *> columns(block.columns());
for (const auto i : ext::range(0, columns.size()))
columns[i] = block.safeGetByPosition(i).column.get();
columns[i] = block.getByPosition(i).column.get();
std::size_t num_rows = 0;
while (row)

View File

@ -1,3 +1,5 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Common/config.h>
#if USE_MYSQL
@ -5,6 +7,7 @@
#include <Dictionaries/MySQLDictionarySource.h>
#include <Dictionaries/MySQLBlockInputStream.h>
#include <Dictionaries/readInvalidateQuery.h>
namespace DB
@ -26,7 +29,8 @@ MySQLDictionarySource::MySQLDictionarySource(const DictionaryStructure & dict_st
sample_block{sample_block},
pool{config, config_prefix},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
load_all_query{query_builder.composeLoadAllQuery()}
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
{
}
@ -41,7 +45,8 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other
sample_block{other.sample_block},
pool{other.pool},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
load_all_query{other.load_all_query}, last_modification{other.last_modification}
load_all_query{other.load_all_query}, last_modification{other.last_modification},
invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response}
{
}
@ -62,7 +67,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & i
}
BlockInputStreamPtr MySQLDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
@ -72,6 +77,15 @@ BlockInputStreamPtr MySQLDictionarySource::loadKeys(
bool MySQLDictionarySource::isModified() const
{
if (!invalidate_query.empty())
{
auto response = doInvalidateQuery(invalidate_query);
if (response == invalidate_query_response)
return false;
invalidate_query_response = response;
return true;
}
if (dont_check_update_time)
return true;
@ -162,7 +176,16 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
return update_time;
}
std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request) const
{
Block sample_block;
ColumnPtr column(std::make_shared<ColumnString>());
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
MySQLBlockInputStream block_input_stream(pool.Get(), request, sample_block, 1);
return readInvalidateQuery(block_input_stream);
}
}
#endif

View File

@ -31,7 +31,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
bool isModified() const override;
@ -42,12 +42,14 @@ public:
std::string toString() const override;
private:
Poco::Logger * log;
static std::string quoteForLike(const std::string s);
LocalDateTime getLastModification() const;
// execute invalidate_query. expects single cell in result
std::string doInvalidateQuery(const std::string & request) const;
Poco::Logger * log;
const DictionaryStructure dict_struct;
const std::string db;
const std::string table;
@ -58,6 +60,8 @@ private:
ExternalQueryBuilder query_builder;
const std::string load_all_query;
LocalDateTime last_modification;
std::string invalidate_query;
mutable std::string invalidate_query_response;
};
}

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <common/logger_useful.h>
#include <ext/range.hpp>
#include <vector>
@ -24,7 +25,8 @@ ODBCBlockInputStream::ODBCBlockInputStream(
statement{(this->session << query_str, Poco::Data::Keywords::now)},
result{statement},
iterator{result.begin()},
max_block_size{max_block_size}
max_block_size{max_block_size},
log(&Logger::get("ODBCBlockInputStream"))
{
if (sample_block.columns() != result.columnCount())
throw Exception{
@ -78,7 +80,7 @@ Block ODBCBlockInputStream::readImpl()
/// cache pointers returned by the calls to getByPosition
std::vector<IColumn *> columns(block.columns());
for (const auto i : ext::range(0, columns.size()))
columns[i] = block.safeGetByPosition(i).column.get();
columns[i] = block.getByPosition(i).column.get();
size_t num_rows = 0;
while (iterator != result.end())
@ -95,11 +97,12 @@ Block ODBCBlockInputStream::readImpl()
insertDefaultValue(columns[idx], *description.sample_columns[idx]);
}
++iterator;
++num_rows;
if (num_rows == max_block_size)
break;
++iterator;
}
return block;

View File

@ -41,6 +41,8 @@ private:
const std::size_t max_block_size;
ExternalResultDescription description;
Poco::Logger * log;
};
}

View File

@ -1,8 +1,11 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Poco/Data/SessionPool.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Dictionaries/ODBCDictionarySource.h>
#include <Dictionaries/ODBCBlockInputStream.h>
#include <common/logger_useful.h>
#include <Dictionaries/readInvalidateQuery.h>
namespace DB
@ -22,7 +25,8 @@ ODBCDictionarySource::ODBCDictionarySource(const DictionaryStructure & dict_stru
where{config.getString(config_prefix + ".where", "")},
sample_block{sample_block},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None}, /// NOTE Better to obtain quoting style via ODBC interface.
load_all_query{query_builder.composeLoadAllQuery()}
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
{
pool = createAndCheckResizePocoSessionPool([&] () { return std::make_shared<Poco::Data::SessionPool>(
config.getString(config_prefix + ".connector", "ODBC"),
@ -40,7 +44,8 @@ ODBCDictionarySource::ODBCDictionarySource(const ODBCDictionarySource & other)
sample_block{other.sample_block},
pool{other.pool},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None},
load_all_query{other.load_all_query}
load_all_query{other.load_all_query},
invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response}
{
}
@ -72,17 +77,12 @@ BlockInputStreamPtr ODBCDictionarySource::loadIds(const std::vector<UInt64> & id
}
BlockInputStreamPtr ODBCDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
return std::make_shared<ODBCBlockInputStream>(pool->get(), query, sample_block, max_block_size);
}
bool ODBCDictionarySource::isModified() const
{
return true;
}
bool ODBCDictionarySource::supportsSelectiveLoad() const
{
return true;
@ -98,5 +98,26 @@ std::string ODBCDictionarySource::toString() const
return "ODBC: " + db + '.' + table + (where.empty() ? "" : ", where: " + where);
}
bool ODBCDictionarySource::isModified() const
{
if (!invalidate_query.empty())
{
auto response = doInvalidateQuery(invalidate_query);
if (invalidate_query_response == response)
return false;
invalidate_query_response = response;
}
return true;
}
std::string ODBCDictionarySource::doInvalidateQuery(const std::string & request) const
{
Block sample_block;
ColumnPtr column(std::make_shared<ColumnString>());
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
ODBCBlockInputStream block_input_stream(pool->get(), request, sample_block, 1);
return readInvalidateQuery(block_input_stream);
}
}

View File

@ -41,7 +41,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override;
const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
bool isModified() const override;
@ -52,6 +52,9 @@ public:
std::string toString() const override;
private:
// execute invalidate_query. expects single cell in result
std::string doInvalidateQuery(const std::string & request) const;
Poco::Logger * log;
const DictionaryStructure dict_struct;
@ -62,6 +65,8 @@ private:
std::shared_ptr<Poco::Data::SessionPool> pool = nullptr;
ExternalQueryBuilder query_builder;
const std::string load_all_query;
std::string invalidate_query;
mutable std::string invalidate_query_response;
using PocoSessionPoolConstructor = std::function<std::shared_ptr<Poco::Data::SessionPool>()>;

View File

@ -51,7 +51,7 @@ TrieDictionary::~TrieDictionary()
#define DECLARE(TYPE)\
void TrieDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
@ -81,7 +81,7 @@ DECLARE(Float64)
#undef DECLARE
void TrieDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyTypes(key_types);
@ -101,7 +101,7 @@ void TrieDictionary::getString(
#define DECLARE(TYPE)\
void TrieDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
@ -129,7 +129,7 @@ DECLARE(Float64)
#undef DECLARE
void TrieDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
validateKeyTypes(key_types);
@ -147,7 +147,7 @@ void TrieDictionary::getString(
#define DECLARE(TYPE)\
void TrieDictionary::get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
@ -175,7 +175,7 @@ DECLARE(Float64)
#undef DECLARE
void TrieDictionary::getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const
{
validateKeyTypes(key_types);
@ -191,7 +191,7 @@ void TrieDictionary::getString(
[&] (const std::size_t) { return StringRef{def}; });
}
void TrieDictionary::has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
void TrieDictionary::has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const
{
validateKeyTypes(key_types);
@ -246,14 +246,16 @@ void TrieDictionary::loadData()
const auto rows = block.rows();
element_count += rows;
const auto key_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, keys_size),
[&] (const std::size_t attribute_idx) {
return block.safeGetByPosition(attribute_idx).column.get();
const auto key_column_ptrs = ext::map<Columns>(ext::range(0, keys_size),
[&] (const std::size_t attribute_idx)
{
return block.safeGetByPosition(attribute_idx).column;
});
const auto attribute_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, attributes_size),
[&] (const std::size_t attribute_idx) {
return block.safeGetByPosition(keys_size + attribute_idx).column.get();
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size),
[&] (const std::size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
for (const auto row_idx : ext::range(0, rows))
@ -373,7 +375,7 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void TrieDictionary::getItemsNumber(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const
{
@ -399,7 +401,7 @@ void TrieDictionary::getItemsNumber(
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void TrieDictionary::getItemsImpl(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const
{
@ -514,7 +516,7 @@ const TrieDictionary::Attribute & TrieDictionary::getAttribute(const std::string
}
template <typename T>
void TrieDictionary::has(const Attribute & attribute, const ConstColumnPlainPtrs & key_columns, PaddedPODArray<UInt8> & out) const
void TrieDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const
{
const auto first_column = key_columns.front();
const auto rows = first_column->size();

View File

@ -68,7 +68,7 @@ public:
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -83,12 +83,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
ColumnString * out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -103,12 +103,12 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const;
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, PaddedPODArray<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -123,10 +123,10 @@ public:
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const;
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
private:
template <typename Value> using ContainerType = std::vector<Value>;
@ -168,14 +168,14 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsNumber(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsImpl(
const Attribute & attribute,
const ConstColumnPlainPtrs & key_columns,
const Columns & key_columns,
ValueSetter && set_value,
DefaultGetter && get_default) const;
@ -188,7 +188,7 @@ private:
const Attribute & getAttribute(const std::string & attribute_name) const;
template <typename T>
void has(const Attribute & attribute, const ConstColumnPlainPtrs & key_columns, PaddedPODArray<UInt8> & out) const;
void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const;
const std::string name;
const DictionaryStructure dict_struct;

View File

@ -0,0 +1,47 @@
#include <Dictionaries/readInvalidateQuery.h>
#include <DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MUCH_COLUMNS;
extern const int TOO_MUCH_ROWS;
extern const int RECEIVED_EMPTY_DATA;
}
std::string readInvalidateQuery(IProfilingBlockInputStream & block_input_stream)
{
block_input_stream.readPrefix();
std::string response;
Block block = block_input_stream.read();
if (!block)
throw Exception("Empty response", ErrorCodes::RECEIVED_EMPTY_DATA);
auto columns = block.columns();
if (columns > 1)
throw Exception("Expected single column in resultset, got " + std::to_string(columns), ErrorCodes::TOO_MUCH_COLUMNS);
auto rows = block.rows();
if (rows == 0)
throw Exception("Expected single row in resultset, got 0", ErrorCodes::RECEIVED_EMPTY_DATA);
if (rows > 1)
throw Exception("Expected single row in resultset, got at least " + std::to_string(rows), ErrorCodes::TOO_MUCH_ROWS);
auto column = block.getByPosition(0).column;
response = column->getDataAt(0).toString();
while ((block = block_input_stream.read()))
{
if (block.rows() > 0)
throw Exception("Expected single row in resultset, got at least " + std::to_string(rows + 1), ErrorCodes::TOO_MUCH_ROWS);
}
block_input_stream.readSuffix();
return response;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <string>
class IProfilingBlockInputStream;
namespace DB
{
// Using in MySQLDictionarySource and ODBCDictionarySource after processing invalidate_query
std::string readInvalidateQuery(IProfilingBlockInputStream & block_input_stream);
}

View File

@ -7,7 +7,7 @@
#include <Columns/ColumnConst.h>
#include <Functions/IFunction.h>
#include <Functions/NumberTraits.h>
#include <Functions/AccurateComparison.h>
#include <Core/AccurateComparison.h>
#include <Core/FieldVisitors.h>

View File

@ -35,28 +35,28 @@ namespace ErrorCodes
}
/** Функции по работе с массивами:
/** Array functions:
*
* array(с1, с2, ...) - создать массив из констант.
* arrayElement(arr, i) - получить элемент массива по индексу.
* Индекс начинается с 1. Также индекс может быть отрицательным - тогда он считается с конца массива.
* has(arr, x) - есть ли в массиве элемент x.
* indexOf(arr, x) - возвращает индекс элемента x (начиная с 1), если он есть в массиве, или 0, если его нет.
* arrayEnumerate(arr) - возаращает массив [1,2,3,..., length(arr)]
* array(c1, c2, ...) - create an array of constants.
* arrayElement(arr, i) - get the array element by index.
* 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.
* 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.
* arrayEnumerate(arr) - Returns the array [1,2,3,..., length(arr)]
*
* arrayUniq(arr) - считает количество разных элементов в массиве,
* arrayUniq(arr1, arr2, ...) - считает количество разных кортежей из элементов на соответствующих позициях в нескольких массивах.
* arrayUniq(arr) - counts the number of different elements in the array,
* arrayUniq(arr1, arr2, ...) - counts the number of different tuples from the elements in the corresponding positions in several arrays.
*
* arrayEnumerateUniq(arr)
* - возаращает массив, параллельный данному, где для каждого элемента указано,
* какой он по счету среди элементов с таким значением.
* Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
* - outputs an array parallel to this, where for each element specified
* how much times this element was encountered before among elements with the same value.
* For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
* arrayEnumerateUniq(arr1, arr2...)
* - для кортежей из элементов на соответствующих позициях в нескольких массивах.
* - for tuples from elements in the corresponding positions in several arrays.
*
* emptyArrayToSingle(arr) - заменить пустые массивы на массивы из одного элемента со значением "по-умолчанию".
* emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value.
*
* arrayReduce('agg', arr1, ...) - применить агрегатную функцию agg к массивам arr1...
* arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...`
*/
@ -142,7 +142,7 @@ private:
template <typename IndexType>
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, ArrayImpl::NullMapBuilder & builder);
/** Для массива кортежей функция вычисляется покомпонентно - для каждого элемента кортежа.
/** For a tuple array, the function is evaluated component-wise for each element of the tuple.
*/
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result);
};
@ -159,7 +159,7 @@ struct IndexToOne
struct IndexIdentity
{
using ResultType = UInt64;
/// Индекс возвращается начиная с единицы.
/// The index is returned starting from 1.
static bool apply(size_t j, ResultType & current) { current = j + 1; return false; }
};
@ -1181,8 +1181,8 @@ public:
};
/// Считает количество разных элементов в массиве, или количество разных кортежей из элементов на соответствующих позициях в нескольких массивах.
/// NOTE Реализация частично совпадает с arrayEnumerateUniq.
/// Counts the number of different elements in the array, or the number of different tuples from the elements at the corresponding positions in several arrays.
/// NOTE The implementation partially matches arrayEnumerateUniq.
class FunctionArrayUniq : public IFunction
{
public:
@ -1199,7 +1199,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private:
/// Изначально выделить кусок памяти для 512 элементов.
/// Initially allocate a piece of memory for 512 elements.
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T>
@ -1239,7 +1239,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private:
/// Изначально выделить кусок памяти для 512 элементов.
/// Initially allocate a piece of memory for 512 elements.
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T>

View File

@ -24,22 +24,23 @@
namespace DB
{
/** Функции кодирования:
/** Coding functions:
*
* IPv4NumToString(num) - См. ниже.
* IPv4StringToNum(string) - Преобразуют, например, '192.168.0.1' в 3232235521 и наоборот.
* IPv4NumToString (num) - See below.
* IPv4StringToNum(string) - Convert, for example, '192.168.0.1' to 3232235521 and vice versa.
*
* hex(x) - Возвращает hex; буквы заглавные; префиксов 0x или суффиксов h нет.
* Для чисел возвращает строку переменной длины - hex в "человеческом" (big endian) формате, с вырезанием старших нулей, но только по целым байтам. Для дат и дат-с-временем - как для чисел.
* Например, hex(257) = '0101'.
* unhex(string) - Возвращает строку, hex от которой равен string с точностью до регистра и отбрасывания одного ведущего нуля.
* Если такой строки не существует, оставляет за собой право вернуть любой мусор.
* 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,
* but only by whole bytes. For dates and datetimes - the same as for numbers.
* 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.
* If such a string does not exist, reserves the right to return any garbage.
*
* bitmaskToArray(x) - Возвращает массив степеней двойки в двоичной записи x. Например, 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;
@ -664,16 +665,16 @@ public:
{
char * begin = out;
/// Запишем все задом наперед.
/// Write everything backwards.
for (size_t offset = 0; offset <= 24; offset += 8)
{
if (offset > 0)
*(out++) = '.';
/// Достаем очередной байт.
/// Get the next byte.
UInt32 value = (ip >> offset) & static_cast<UInt32>(255);
/// Быстрее, чем sprintf.
/// Faster than sprintf.
if (value == 0)
{
*(out++) = '0';
@ -688,7 +689,7 @@ public:
}
}
/// И развернем.
/// And reverse.
std::reverse(begin, out);
*(out++) = '\0';
@ -708,7 +709,7 @@ public:
ColumnString::Chars_t & vec_res = col_res->getChars();
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
vec_res.resize(vec_in.size() * INET_ADDRSTRLEN); /// самое длинное значение: 255.255.255.255\0
vec_res.resize(vec_in.size() * INET_ADDRSTRLEN); /// the longest value is: 255.255.255.255\0
offsets_res.resize(vec_in.size());
char * begin = reinterpret_cast<char *>(&vec_res[0]);
char * pos = begin;
@ -847,16 +848,16 @@ public:
for (auto i = 0; i < 3; ++i)
*(out++) = 'x';
/// Запишем все задом наперед.
/// Write everything backwards.
for (size_t offset = 8; offset <= 24; offset += 8)
{
if (offset > 0)
*(out++) = '.';
/// Достаем очередной байт.
/// Get the next byte.
UInt32 value = (ip >> offset) & static_cast<UInt32>(255);
/// Быстрее, чем sprintf.
/// Faster than sprintf.
if (value == 0)
{
*(out++) = '0';
@ -871,7 +872,7 @@ public:
}
}
/// И развернем.
/// And reverse.
std::reverse(begin, out);
*(out++) = '\0';
@ -891,7 +892,7 @@ public:
ColumnString::Chars_t & vec_res = col_res->getChars();
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
vec_res.resize(vec_in.size() * INET_ADDRSTRLEN); /// самое длинное значение: 255.255.255.255\0
vec_res.resize(vec_in.size() * INET_ADDRSTRLEN); /// the longest value is: 255.255.255.255\0
offsets_res.resize(vec_in.size());
char * begin = reinterpret_cast<char *>(&vec_res[0]);
char * pos = begin;
@ -1015,16 +1016,16 @@ public:
/// mapping of digits up to base 16
static char digits[] = "0123456789ABCDEF";
/// Запишем все задом наперед.
/// Write everything backwards.
for (size_t offset = 0; offset <= 40; offset += 8)
{
if (offset > 0)
*(out++) = ':';
/// Достаем очередной байт.
/// Get the next byte.
UInt64 value = (mac >> offset) & static_cast<UInt64>(255);
/// Быстрее, чем sprintf.
/// Faster than sprintf.
if (value < 16)
{
*(out++) = '0';
@ -1043,7 +1044,7 @@ public:
}
}
/// И развернем.
/// And reverse.
std::reverse(begin, out);
*(out++) = '\0';
@ -1063,7 +1064,7 @@ public:
ColumnString::Chars_t & vec_res = col_res->getChars();
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
vec_res.resize(vec_in.size() * 18); /// самое длинное значение: xx:xx:xx:xx:xx:xx\0
vec_res.resize(vec_in.size() * 18); /// the longest value is: xx:xx:xx:xx:xx:xx\0
offsets_res.resize(vec_in.size());
char * begin = reinterpret_cast<char *>(&vec_res[0]);
char * pos = begin;
@ -1601,7 +1602,7 @@ public:
{
UInt8 byte = static_cast<UInt8>((x >> offset) & 255);
/// Ведущие нули.
/// Leading zeros.
if (byte == 0 && !was_nonzero && offset)
continue;
@ -1635,7 +1636,7 @@ public:
size_t pos = 0;
for (size_t i = 0; i < size; ++i)
{
/// Ручной экспоненциальный рост, чтобы не полагаться на линейное амортизированное время работы resize (его никто не гарантирует).
/// Manual exponential growth, so as not to rely on the linear amortized work time of `resize` (no one guarantees it).
if (pos + MAX_UINT_HEX_LENGTH > out_vec.size())
out_vec.resize(out_vec.size() * 2 + MAX_UINT_HEX_LENGTH);
@ -1724,7 +1725,7 @@ public:
std::string res(src.size() * 2, '\0');
char * pos = &res[0];
const UInt8 * src_ptr = reinterpret_cast<const UInt8 *>(src.c_str());
/// Запишем ноль в res[res.size()]. Начиная с C++11, это корректно.
/// Let's write zero into res[res.size()]. Starting with C++ 11, this is correct.
executeOneString(src_ptr, src_ptr + src.size(), pos);
col_res = std::make_shared<ColumnConstString>(col_const_in->size(), res);

View File

@ -17,8 +17,8 @@
#include <Functions/FunctionsLogical.h>
#include <Functions/IFunction.h>
#include <Functions/AccurateComparison.h>
#include <Core/AccurateComparison.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
@ -29,20 +29,20 @@
namespace DB
{
/** Функции сравнения: ==, !=, <, >, <=, >=.
* Функции сравнения возвращают всегда 0 или 1 (UInt8).
/** Comparison functions: ==, !=, <, >, <=, >=.
* The comparison functions always return 0 or 1 (UInt8).
*
* Сравнивать можно следующие типы:
* - числа;
* - строки и фиксированные строки;
* - даты;
* - даты-с-временем;
* внутри каждой группы, но не из разных групп;
* - кортежи (сравнение лексикографическое).
* You can compare the following types:
* - numbers;
* - strings and fixed strings;
* - dates;
* - datetimes;
* within each group, but not from different groups;
* - tuples (lexicographic comparison).
*
* Исключение: можно сравнивать дату и дату-с-временем с константной строкой. Пример: EventDate = '2015-01-01'.
* Exception: You can compare the date and datetime with a constant string. Example: EventDate = '2015-01-01'.
*
* TODO Массивы.
* TODO Arrays.
*/
template <typename A, typename B> struct EqualsOp
@ -94,9 +94,9 @@ struct NumComparisonImpl
/// If you don't specify NO_INLINE, the compiler will inline this function, but we don't need this as this function contains tight loop inside.
static void NO_INLINE vector_vector(const PaddedPODArray<A> & a, const PaddedPODArray<B> & b, PaddedPODArray<UInt8> & c)
{
/** GCC 4.8.2 векторизует цикл только если его записать в такой форме.
* В данном случае, если сделать цикл по индексу массива (код будет выглядеть проще),
* цикл не будет векторизовываться.
/** GCC 4.8.2 vectorizes a loop only if it is written in this form.
* In this case, if you loop through the array index (the code will look simpler),
* the loop will not be vectorized.
*/
size_t size = a.size();
@ -178,7 +178,7 @@ struct StringComparisonImpl
{
if (i == 0)
{
/// Завершающий ноль в меньшей по длине строке входит в сравнение.
/// The trailing zero in the smaller string is included in the comparison.
c[i] = Op::apply(memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0])), 0);
}
else
@ -742,7 +742,7 @@ private:
Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped,
const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, bool right_is_num)
{
/// Уже не такой и особый случай - сравнение дат, дат-с-временем и перечислений со строковой константой.
/// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant.
const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped;
const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped;
const IDataType * number_type = left_is_num ? left_type.get() : right_type.get();
@ -817,7 +817,7 @@ private:
void executeTuple(Block & block, size_t result, const IColumn * c0, const IColumn * c1)
{
/** Сравнивать кортежи будем лексикографически. Это делается следующим образом:
/** We will lexicographically compare the tuples. This is done as follows:
* x == y : x1 == y1 && x2 == y2 ...
* x != y : x1 != y1 || x2 != y2 ...
*
@ -825,7 +825,7 @@ private:
* x > y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn > yn))
* x <= y: x1 < y1 || (x1 == y1 && (x2 < y2 || (x2 == y2 ... && xn <= yn))
*
* Рекурсивная запись:
* Recursive record:
* x <= y: x1 < y1 || (x1 == y1 && x_tail <= y_tail)
*
* x >= y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn >= yn))
@ -874,12 +874,12 @@ private:
tmp_block.insert(x->getData().safeGetByPosition(i));
tmp_block.insert(y->getData().safeGetByPosition(i));
/// Сравнение элементов.
/// Comparison of the elements.
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
func_compare.execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2);
}
/// Логическая свёртка.
/// Logical convolution.
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
ColumnNumbers convolution_args(tuple_size);
@ -901,7 +901,7 @@ private:
Block tmp_block;
/// Попарное сравнение на неравенство всех элементов; на равенство всех элементов кроме последнего.
/// Pairwise comparison of the inequality of all elements; on the equality of all elements except the last.
for (size_t i = 0; i < tuple_size; ++i)
{
tmp_block.insert(x->getData().safeGetByPosition(i));
@ -921,7 +921,7 @@ private:
func_compare_tail.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2);
}
/// Комбинирование. Сложный код - сделайте рисунок. Можно заменить на рекурсивное сравнение кортежей.
/// Combination. Complex code - make a drawing. It can be replaced by a recursive comparison of tuples.
size_t i = tuple_size - 1;
while (i > 0)
{
@ -970,7 +970,7 @@ public:
size_t getNumberOfArguments() const override { return 2; }
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
bool left_is_date = false;
@ -1014,7 +1014,7 @@ public:
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_date && right_is_date)
|| (left_is_date && right_is_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_date_time && right_is_date_time)
|| (left_is_date_time && right_is_string)

View File

@ -19,9 +19,9 @@
namespace DB
{
/** Функция выбора по условию: if(cond, then, else).
/** Selection function by condition: if(cond, then, else).
* cond - UInt8
* then, else - числовые типы, для которых есть общий тип, либо даты, даты-с-временем, либо строки, либо массивы таких типов.
* then, else - numeric types for which there is a general type, or dates, datetimes, or strings, or arrays of these types.
*/
@ -610,9 +610,9 @@ public:
};
/** Реализация для массивов строк.
* NOTE: Код слишком сложный, потому что он работает в внутренностями массивов строк.
* NOTE: Массивы из FixedString не поддерживаются.
/** Implementation for string arrays.
* NOTE: The code is too complex because it works with the internals of the arrays of strings.
* NOTE: Arrays of FixedString are not supported.
*/
struct StringArrayIfImpl
{
@ -666,7 +666,7 @@ struct StringArrayIfImpl
for (size_t j = 0; j < array_size; ++j)
{
const String & str = from_data[j].get<const String &>();
size_t string_size = str.size() + 1; /// Включая 0 на конце.
size_t string_size = str.size() + 1; /// Including 0 at the end.
to_data.resize(to_string_prev_offset + string_size);
memcpy(&to_data[to_string_prev_offset], str.data(), string_size);
@ -1121,7 +1121,7 @@ private:
{
if (col_then_fixed && col_else_fixed)
{
/// Результат - FixedString.
/// The result is FixedString.
if (col_then_fixed->getN() != col_else_fixed->getN())
throw Exception("FixedString columns as 'then' and 'else' arguments of function 'if' has different sizes", ErrorCodes::ILLEGAL_COLUMN);
@ -1142,7 +1142,7 @@ private:
}
else
{
/// Результат - String.
/// The result is String.
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
block.safeGetByPosition(result).column = col_res;
@ -1582,7 +1582,7 @@ public:
bool hasSpecialSupportForNulls() const override { return true; }
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
bool cond_is_null = arguments[0]->isNull();
@ -1640,7 +1640,7 @@ public:
}
else if (type_arr1 && type_arr2)
{
/// NOTE Сообщения об ошибках будут относится к типам элементов массивов, что немного некорректно.
/// NOTE Error messages will refer to the types of array elements, which is slightly incorrect.
return std::make_shared<DataTypeArray>(getReturnTypeImpl({arguments[0], type_arr1->getNestedType(), type_arr2->getNestedType()}));
}
else if (type_tuple1 && type_tuple2)

View File

@ -23,7 +23,7 @@
namespace DB
{
/** Функции работы с датой и временем.
/** Functions for working with date and time.
*
* toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond,
* toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfFiveMinute
@ -31,43 +31,43 @@ namespace DB
* now
* TODO: makeDate, makeDateTime
*
* (toDate - расположена в файле FunctionsConversion.h)
* (toDate - located in FunctionConversion.h file)
*
* Возвращаемые типы:
* Return types:
* toYear -> UInt16
* toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond -> UInt8
* toMonday, toStartOfMonth, toStartOfYear -> Date
* toStartOfMinute, toStartOfHour, toTime, now -> DateTime
*
* А также:
* And also:
*
* timeSlot(EventTime)
* - округляет время до получаса.
* - rounds the time to half an hour.
*
* timeSlots(StartTime, Duration)
* - для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд,
* возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала.
* Например, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')].
* Это нужно для поиска хитов, входящих в соответствующий визит.
* - for the time interval beginning at `StartTime` and continuing `Duration` seconds,
* returns an array of time points, consisting of rounding down to half an hour of points from this interval.
* For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')].
* This is necessary to search for hits that are part of the corresponding visit.
*/
#define TIME_SLOT_SIZE 1800
/** Всевозможные преобразования.
* Представляют собой две функции - от даты-с-временем (UInt32) и от даты (UInt16).
/** Extra transformations.
* Represents two functions - from datetime (UInt32) and from date (UInt16).
*
* Также для преобразования T определяется "фактор-преобразование" F.
* Это такое преобразование F, что его значение идентифицирует область монотонности T
* (при фиксированном значении F, преобразование T является монотонным).
* Also, the "factor transformation" F is defined for the T transformation.
* This is a transformation of F such that its value identifies the region of monotonicity T
* (for a fixed value of F, the transformation T is monotonic).
*
* Или, образно, если T аналогично взятию остатка от деления, то F аналогично делению.
* Or, figuratively, if T is similar to taking the remainder of division, then F is similar to division.
*
* Пример: для преобразования T "получить номер дня в месяце" (2015-02-03 -> 3),
* фактор-преобразованием F является "округлить до месяца" (2015-02-03 -> 2015-02-01).
* Example: to convert T "get the day number in the month" (2015-02-03 -> 3),
* factor-transformation F is "round to the nearest month" (2015-02-03 -> 2015-02-01).
*/
/// Это фактор-преобразование будет говорить, что функция монотонна всюду.
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return 0; }
@ -161,7 +161,7 @@ struct ToStartOfYearImpl
struct ToTimeImpl
{
/// При переводе во время, дату будем приравнивать к 1970-01-02.
/// When transforming to time, the date will be equated to 1970-01-02.
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{
return time_zone.toTime(t) + 86400;
@ -554,13 +554,13 @@ public:
if (std::is_same<typename Transform::FactorTransform, ZeroTransform>::value)
return is_monotonic;
/// Этот метод вызывается только если у функции один аргумент. Поэтому, нас пока не волнует не-локальная тайм-зона.
/// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone.
const DateLUTImpl & date_lut = DateLUT::instance();
if (left.isNull() || right.isNull())
return is_not_monotonic;
/// Функция монотонна на отрезке [left, right], если фактор-преобразование возвращает для них одинаковые значения.
/// The function is monotonous on the [left, right] segment, if the factor transformation returns the same values for them.
if (typeid_cast<const DataTypeDate *>(&type))
{
@ -578,7 +578,7 @@ public:
};
/// Получить текущее время. (Оно - константа, вычисляется один раз за весь запрос.)
/// Get the current time. (It is a constant, it is evaluated once for the entire query.)
class FunctionNow : public IFunction
{
public:

View File

@ -160,10 +160,7 @@ private:
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
const ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(),
[](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col_materialized).getColumns();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = std::make_shared<ColumnUInt8>(key_col_with_type.column->size());
@ -370,10 +367,7 @@ private:
{
const ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(),
[](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col_materialized).getColumns();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = std::make_shared<ColumnString>();
@ -681,9 +675,7 @@ private:
const ColumnPtr key_col_materialized = key_col.convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col_materialized).getColumns();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = std::make_shared<ColumnString>();
@ -724,7 +716,7 @@ template <> struct DictGetTraits<DATA_TYPE>\
}\
template <typename DictionaryType>\
static void get(\
const DictionaryType * dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DictionaryType * dict, const std::string & name, const Columns & key_columns,\
const DataTypes & key_types, PaddedPODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, out);\
@ -745,7 +737,7 @@ template <> struct DictGetTraits<DATA_TYPE>\
}\
template <typename DictionaryType, typename DefaultsType>\
static void getOrDefault(\
const DictionaryType * dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DictionaryType * dict, const std::string & name, const Columns & key_columns,\
const DataTypes & key_types, const DefaultsType & def, PaddedPODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, def, out);\
@ -934,10 +926,7 @@ private:
{
const ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(),
[](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col_materialized).getColumns();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = std::make_shared<ColumnVector<Type>>(key_columns.front()->size());
@ -1290,9 +1279,7 @@ private:
const ColumnPtr key_col_materialized = key_col.convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col_materialized).getColumns();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
/// @todo detect when all key columns are constant

View File

@ -31,21 +31,21 @@
namespace DB
{
/** Функции хэширования.
/** Hashing functions.
*
* Половинка MD5:
* Half MD5:
* halfMD5: String -> UInt64
*
* Более быстрая криптографическая хэш-функция:
* A faster cryptographic hash function:
* sipHash64: String -> UInt64
*
* Быстрая некриптографическая хэш функция для строк:
* Fast non-cryptographic hash function for strings:
* cityHash64: String -> UInt64
*
* Некриптографический хеш от кортежа значений любых типов (использует cityHash64 для строк и intHash64 для чисел):
* A non-cryptographic hash from a tuple of values of any types (uses cityHash64 for strings and intHash64 for numbers):
* cityHash64: any* -> UInt64
*
* Быстрая некриптографическая хэш функция от любого целого числа:
* Fast non-cryptographic hash function from any integer:
* intHash32: number -> UInt32
* intHash64: number -> UInt64
*
@ -66,7 +66,7 @@ struct HalfMD5Impl
MD5_Update(&ctx, reinterpret_cast<const unsigned char *>(begin), size);
MD5_Final(buf.char_data, &ctx);
return Poco::ByteOrder::flipBytes(buf.uint64_data); /// Совместимость с существующим кодом.
return Poco::ByteOrder::flipBytes(buf.uint64_data); /// Compatibility with existing code.
}
};
@ -151,7 +151,7 @@ struct IntHash32Impl
static UInt32 apply(UInt64 x)
{
/// seed взят из /dev/urandom. Он позволяет избежать нежелательных зависимостей с хэшами в разных структурах данных.
/// seed is taken from /dev/urandom. It allows you to avoid undesirable dependencies with hashes in different data structures.
return intHash32<0x75D9543DE018BF45ULL>(x);
}
};
@ -371,8 +371,8 @@ template <>
UInt64 toInteger<Float64>(Float64 x);
/** Используются хэш-функции под названием CityHash, FarmHash, MetroHash.
* В связи с этим, этот шаблон назван со словами NeighbourhoodHash.
/** We use hash functions called CityHash, FarmHash, MetroHash.
* In this regard, this template is named with the words `NeighborhoodHash`.
*/
template <typename Impl>
class FunctionNeighbourhoodHash64 : public IFunction
@ -507,7 +507,7 @@ private:
}
else if (const ColumnConstArray * col_from = typeid_cast<const ColumnConstArray *>(column))
{
/// NOTE: тут, конечно, можно обойтись без материалиации столбца.
/// NOTE: here, of course, you can do without the materialization of the column.
ColumnPtr full_column = col_from->convertToFullColumn();
executeArray<first>(type, &*full_column, vec_to);
}
@ -644,15 +644,15 @@ struct URLHierarchyHashImpl
{
auto pos = begin;
/// Распарсим всё, что идёт до пути
/// Let's parse everything that goes before the path
/// Предположим, что протокол уже переведён в нижний регистр.
/// Suppose that the protocol has already been changed to lowercase.
while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9')))
++pos;
/** Будем вычислять иерархию только для URL-ов, в которых есть протокол, и после него идут два слеша.
* (http, file - подходят, mailto, magnet - не подходят), и после двух слешей ещё хоть что-нибудь есть
* Для остальных просто вернём полный URL как единственный элемент иерархии.
/** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes.
* (http, file - fit, mailto, magnet - do not fit), and after two slashes there is still something
* For the rest, simply return the full URL as the only element of the hierarchy.
*/
if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end))
{
@ -660,7 +660,7 @@ struct URLHierarchyHashImpl
return 0 == level ? pos - begin : 0;
}
/// Доменом для простоты будем считать всё, что после протокола и двух слешей, до следующего слеша или до ? или до #
/// The domain for simplicity is everything that after the protocol and the two slashes, until the next slash or before `?` or `#`
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
@ -674,7 +674,7 @@ struct URLHierarchyHashImpl
while (current_level != level && pos < end)
{
/// Идём до следующего / или ? или #, пропуская все те, что вначале.
/// We go to the next `/` or `?` or `#`, skipping all at the beginning.
while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos == end)

View File

@ -20,24 +20,24 @@ namespace ErrorCodes
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
}
/** Функции высшего порядка для массивов:
/** Higher-order functions for arrays:
*
* arrayMap(x1,...,xn -> expression, array1,...,arrayn) - применить выражение к каждому элементу массива (или набора параллельных массивов).
* arrayFilter(x -> predicate, array) - оставить в массиве только элементы, для которых выражение истинно.
* arrayCount(x1,...,xn -> expression, array1,...,arrayn) - для скольки элементов массива выражение истинно.
* arrayExists(x1,...,xn -> expression, array1,...,arrayn) - истинно ли выражение для хотя бы одного элемента массива.
* arrayAll(x1,...,xn -> expression, array1,...,arrayn) - истинно ли выражение для всех элементов массива.
* arrayMap(x1,...,xn -> expression, array1,...,arrayn) - apply the expression to each element of the array (or set of parallel arrays).
* arrayFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true.
* arrayCount(x1,...,xn -> expression, array1,...,arrayn) - for how many elements of the array the expression is true.
* arrayExists(x1,...,xn -> expression, array1,...,arrayn) - is the expression true for at least one array element.
* arrayAll(x1,...,xn -> expression, array1,...,arrayn) - is the expression true for all elements of the array.
*
* Для функций arrayCount, arrayExists, arrayAll доступна еще перегрузка вида f(array), которая работает так же, как f(x -> x, array).
* For functions arrayCount, arrayExists, arrayAll, an overload of the form f(array) is available, which works in the same way as f(x -> x, array).
*/
struct ArrayMapImpl
{
/// true, если выражение (для перегрузки f(expression, arrays)) или массив (для f(array)) должно быть булевым.
/// true if the expression (for an overload of f(expression, arrays)) or an array (for f(array)) should be boolean.
static bool needBoolean() { return false; }
/// true, если перегрузка f(array) недоступна.
/// true if the f(array) overload is unavailable.
static bool needExpression() { return true; }
/// true, если массив должен быть ровно один.
/// true if the array must be exactly one.
static bool needOneArray() { return false; }
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & array_element)
@ -64,7 +64,7 @@ struct ArrayFilterImpl
return std::make_shared<DataTypeArray>(array_element);
}
/// Если массивов несколько, сюда передается первый.
/// If there are several arrays, the first one is passed here.
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{
const ColumnUInt8 * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
@ -610,8 +610,8 @@ public:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
/// Вызывается, если хоть один агрумент функции - лямбда-выражение.
/// Для аргументов-лямбда-выражений определяет типы аргументов этих выражений.
/// Called if at least one function argument is a lambda expression.
/// For argument-lambda expressions, it defines the types of arguments of these expressions.
void getLambdaArgumentTypesImpl(DataTypes & arguments) const override
{
if (arguments.size() < 1)
@ -679,7 +679,7 @@ public:
throw Exception("Type of first argument for function " + getName() + " must be an expression.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// Типы остальных аргументов уже проверены в getLambdaArgumentTypes.
/// The types of the remaining arguments are already checked in getLambdaArgumentTypes.
DataTypePtr return_type = expression->getReturnType();
if (Impl::needBoolean() && !typeid_cast<const DataTypeUInt8 *>(&*return_type))
@ -735,9 +735,9 @@ public:
throw Exception("Column of first argument for function " + getName() + " must be an expression.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// Типы остальных аргументов уже проверены в getLambdaArgumentTypes.
/// The types of the remaining arguments are already checked in getLambdaArgumentTypes.
/// Попросим добавить в блок все столбцы, упоминаемые в выражении, размноженные в массив, параллельный обрабатываемому.
/// Let's add to the block all the columns mentioned in the expression, multiplied into an array parallel to the one being processed.
const ExpressionActions & expression = *column_expression->getExpression();
const NamesAndTypesList & required_columns = expression.getRequiredColumnsWithTypes();
@ -803,7 +803,7 @@ public:
ColumnPtr column_first_array_ptr;
const ColumnArray * column_first_array = nullptr;
/// Положим в блок аргументы выражения.
/// Put the expression arguments in the block.
for (size_t i = 0; i < expression_arguments.size(); ++i)
{
@ -828,7 +828,7 @@ public:
}
else
{
/// Первое условие - оптимизация: не сравнивать данные, если указатели равны.
/// The first condition is optimization: do not compare data if the pointers are equal.
if (column_array->getOffsetsColumn() != offsets_column
&& column_array->getOffsets() != typeid_cast<const ColumnArray::ColumnOffsets_t &>(*offsets_column).getData())
throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
@ -848,7 +848,7 @@ public:
argument_names.insert(argument_name);
}
/// Положим в блок все нужные столбцы, размноженные по размерам массивов.
/// Put all the necessary columns multiplied by the sizes of arrays into the block.
Names required_columns = expression.getRequiredColumns();
size_t prerequisite_index = 0;

View File

@ -11,8 +11,8 @@
namespace DB
{
/** Функции - логические связки: and, or, not, xor.
* Принимают любые числовые типы, возвращают UInt8, содержащий 0 или 1.
/** Functions are logical links: and, or, not, xor.
* Accept any numeric types, return a UInt8 containing 0 or 1.
*/
template<typename B>
@ -91,7 +91,7 @@ using UInt8ColumnPtrs = std::vector<const ColumnUInt8 *>;
template <typename Op, size_t N>
struct AssociativeOperationImpl
{
/// Выбрасывает N последних столбцов из in (если их меньше, то все) и кладет в result их комбинацию.
/// Erases the N last columns from `in` (if there are less, then all) and puts into `result` their combination.
static void execute(UInt8ColumnPtrs & in, UInt8Container & result)
{
if (N > in.size())
@ -113,11 +113,11 @@ struct AssociativeOperationImpl
const UInt8Container & vec;
AssociativeOperationImpl<Op, N - 1> continuation;
/// Запоминает последние N столбцов из in.
/// Remembers the last N columns from `in`.
AssociativeOperationImpl(UInt8ColumnPtrs & in)
: vec(in[in.size() - N]->getData()), continuation(in) {}
/// Возвращает комбинацию значений в i-й строке всех столбцов, запомненных в конструкторе.
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
inline UInt8 apply(size_t i) const
{
if (Op::isSaturable())
@ -252,7 +252,7 @@ public:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
@ -281,11 +281,11 @@ public:
}
size_t n = in[0]->size();
/// Скомбинируем все константные столбцы в одно значение.
/// Combine all constant columns into a single value.
UInt8 const_val = 0;
bool has_consts = extractConstColumns(in, const_val);
// Если это значение однозначно определяет результат, вернем его.
// If this value uniquely determines the result, return it.
if (has_consts && (in.empty() || Impl<UInt8>::apply(const_val, 0) == Impl<UInt8>::apply(const_val, 1)))
{
if (!in.empty())
@ -295,7 +295,7 @@ public:
return;
}
/// Если это значение - нейтральный элемент, забудем про него.
/// If this value is a neutral element, let's forget about it.
if (has_consts && Impl<UInt8>::apply(const_val, 0) == 0 && Impl<UInt8>::apply(const_val, 1) == 1)
has_consts = false;
@ -313,8 +313,8 @@ public:
vec_res.resize(n);
}
/// Разделим входные столбцы на UInt8 и остальные. Первые обработаем более эффективно.
/// col_res в каждый момент будет либо находится в конце uint8_in, либо не содержаться в uint8_in.
/// Divide the input columns into UInt8 and the rest. The first will be processed more efficiently.
/// col_res at each moment will either be at the end of uint8_in, or not contained in uint8_in.
UInt8ColumnPtrs uint8_in;
ColumnPlainPtrs other_in;
for (IColumn * column : in)
@ -325,7 +325,7 @@ public:
other_in.push_back(column);
}
/// Нужен хотя бы один столбец в uint8_in, чтобы было с кем комбинировать столбцы из other_in.
/// You need at least one column in uint8_in, so that you can combine columns from other_in.
if (uint8_in.empty())
{
if (other_in.empty())
@ -336,16 +336,16 @@ public:
uint8_in.push_back(col_res.get());
}
/// Эффективно скомбинируем все столбцы правильного типа.
/// Effectively combine all the columns of the correct type.
while (uint8_in.size() > 1)
{
/// При большом размере блока объединять по 6 толбцов за проход быстрее всего.
/// При маленьком - чем больше, тем быстрее.
/// With a large block size, combining 6 columns per pass is the fastest.
/// When small - more, is faster.
AssociativeOperationImpl<Impl<UInt8>, 10>::execute(uint8_in, vec_res);
uint8_in.push_back(col_res.get());
}
/// По одному добавим все столбцы неправильного типа.
/// Add all the columns of the wrong type one at a time.
while (!other_in.empty())
{
executeUInt8Other(uint8_in[0]->getData(), other_in.back(), vec_res);
@ -353,7 +353,7 @@ public:
uint8_in[0] = col_res.get();
}
/// Такое возможно, если среди аргументов ровно один неконстантный, и он имеет тип UInt8.
/// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8.
if (uint8_in[0] != col_res.get())
{
vec_res.assign(uint8_in[0]->getData());

View File

@ -12,24 +12,24 @@
namespace DB
{
/** Функции генерации псевдослучайных чисел.
* Функция может быть вызвана без аргументов или с одним аргументом.
* Аргумент игнорируется и служит лишь для того, чтобы несколько вызовов одной функции считались разными и не склеивались.
/** Pseudo-random number generation functions.
* The function can be called without arguments or with one argument.
* The argument is ignored and only serves to ensure that several calls to one function are considered different and do not stick together.
*
* Пример:
* SELECT rand(), rand() - выдаст два одинаковых столбца.
* SELECT rand(1), rand(2) - выдаст два разных столбца.
* Example:
* SELECT rand(), rand() - will output two identical columns.
* SELECT rand(1), rand(2) - will output two different columns.
*
* Некриптографические генераторы:
* Non-cryptographic generators:
*
* rand - linear congruental generator 0 .. 2^32 - 1.
* rand64 - комбинирует несколько значений rand, чтобы получить значения из диапазона 0 .. 2^64 - 1.
* rand64 - combines several rand values to get values from the range 0 .. 2^64 - 1.
*
* randConstant - служебная функция, выдаёт константный столбец со случайным значением.
* randConstant - service function, produces a constant column with a random value.
*
* В качестве затравки используют время.
* Замечание: переинициализируется на каждый блок.
* Это значит, что таймер должен быть достаточного разрешения, чтобы выдавать разные значения на каждый блок.
* The time is used as the seed.
* Note: it is reinitialized for each block.
* This means that the timer must be of sufficient resolution to give different values to each block.
*/
namespace detail
@ -42,11 +42,11 @@ namespace detail
struct LinearCongruentialGenerator
{
/// Константы из man lrand48_r.
/// Constants from man lrand48_r.
static constexpr UInt64 a = 0x5DEECE66D;
static constexpr UInt64 c = 0xB;
/// А эта - из head -c8 /dev/urandom | xxd -p
/// And this is from `head -c8 /dev/urandom | Xxd -p`
UInt64 current = 0x09826f4a081cee35ULL;
LinearCongruentialGenerator() {}
@ -194,7 +194,7 @@ class FunctionRandomConstant : public IFunction
private:
using ToType = typename Impl::ReturnType;
/// Значение одно для разных блоков.
/// The value is one for different blocks.
bool is_initialized = false;
ToType value;

View File

@ -15,19 +15,19 @@
namespace DB
{
/** Функции округления:
* roundToExp2 - вниз до ближайшей степени двойки;
* roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
* roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45, 55.
/** Rounding Functions:
* roundToExp2 - down to the nearest power of two;
* roundDuration - down to the nearest of: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
* roundAge - down to the nearest of: 0, 18, 25, 35, 45, 55.
*
* round(x, N) - арифметическое округление (N = 0 по умолчанию).
* ceil(x, N) - наименьшее число, которое не меньше x (N = 0 по умолчанию).
* floor(x, N) - наибольшее число, которое не больше x (N = 0 по умолчанию).
* round(x, N) - arithmetic rounding (N = 0 by default).
* ceil(x, N) is the smallest number that is at least x (N = 0 by default).
* floor(x, N) is the largest number that is not greater than x (N = 0 by default).
*
* Значение параметра N:
* - N > 0: округлять до числа с N десятичными знаками после запятой
* - N < 0: окурглять до целого числа с N нулевыми знаками
* - N = 0: округлять до целого числа
* The value of the parameter N:
* - N > 0: round to the number with N decimal places after the decimal point
* - N < 0: round to an integer with N zero characters
* - N = 0: round to an integer
*/
template<typename A>
@ -106,8 +106,8 @@ struct RoundAgeImpl
}
};
/** Быстрое вычисление остатка от деления для применения к округлению целых чисел.
* Без проверки, потому что делитель всегда положительный.
/** Quick calculation of the remainder of the division to apply to the rounding of integers.
* Without verification, because the divisor is always positive.
*/
template<typename T, typename Enable = void>
struct FastModulo;
@ -161,14 +161,14 @@ public:
}
};
/** Этот параметр контролирует поведение функций округления.
/** This parameter controls the behavior of the rounding functions.
*/
enum ScaleMode
{
PositiveScale, // округлять до числа с N десятичными знаками после запятой
NegativeScale, // окурглять до целого числа с N нулевыми знаками
ZeroScale, // округлять до целого числа
NullScale // возвращать нулевое значение
PositiveScale, // round to a number with N decimal places after the decimal point
NegativeScale, // round to an integer with N zero characters
ZeroScale, // round to an integer
NullScale // return zero value
};
#if !defined(_MM_FROUND_NINT)
@ -177,7 +177,7 @@ enum ScaleMode
#define _MM_FROUND_CEIL 2
#endif
/** Реализация низкоуровневых функций округления для целочисленных значений.
/** Implementing low-level rounding functions for integer values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
struct IntegerRoundingComputation;
@ -283,7 +283,7 @@ public:
static const size_t data_count = 4;
protected:
/// Предотвратить появление отрицательных нолей определённых в стандарте IEEE-754.
/// Prevent the appearance of negative zeros defined in the IEEE-754 standard.
static inline void normalize(__m128 & val, const __m128 & mask)
{
__m128 mask1 = _mm_cmpeq_ps(val, getZero());
@ -321,7 +321,7 @@ public:
static const size_t data_count = 2;
protected:
/// Предотвратить появление отрицательных нолей определённых в стандарте IEEE-754.
/// Prevent the occurrence of negative zeros defined in the IEEE-754 standard.
static inline void normalize(__m128d & val, const __m128d & mask)
{
__m128d mask1 = _mm_cmpeq_pd(val, getZero());
@ -351,7 +351,7 @@ protected:
}
};
/** Реализация низкоуровневых функций округления для значений с плавающей точкой.
/** Implementation of low-level round-off functions for floating-point values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
class FloatRoundingComputation;
@ -372,7 +372,7 @@ public:
__m128 val = _mm_loadu_ps(in);
__m128 mask = _mm_cmplt_ps(val, getZero());
/// Алгоритм округления.
/// Rounding algorithm.
val = _mm_mul_ps(val, scale);
val = _mm_round_ps(val, rounding_mode);
val = _mm_div_ps(val, scale);
@ -398,20 +398,20 @@ public:
__m128 val = _mm_loadu_ps(in);
__m128 mask = _mm_cmplt_ps(val, getZero());
/// Превратить отрицательные значения в положительные.
/// Turn negative values into positive values.
__m128 factor = _mm_cmpge_ps(val, getZero());
factor = _mm_min_ps(factor, getTwo());
factor = _mm_sub_ps(factor, getOne());
val = _mm_mul_ps(val, factor);
/// Алгоритм округления.
/// Rounding algorithm.
val = _mm_div_ps(val, scale);
__m128 res = _mm_cmpge_ps(val, getOneTenth());
val = _mm_round_ps(val, rounding_mode);
val = _mm_mul_ps(val, scale);
val = _mm_and_ps(val, res);
/// Вернуть настоящие знаки всех значений.
/// Return the real signs of all values.
val = _mm_mul_ps(val, factor);
normalize(val, mask);
@ -463,7 +463,7 @@ public:
__m128d val = _mm_loadu_pd(in);
__m128d mask = _mm_cmplt_pd(val, getZero());
/// Алгоритм округления.
/// Rounding algorithm.
val = _mm_mul_pd(val, scale);
val = _mm_round_pd(val, rounding_mode);
val = _mm_div_pd(val, scale);
@ -489,20 +489,20 @@ public:
__m128d val = _mm_loadu_pd(in);
__m128d mask = _mm_cmplt_pd(val, getZero());
/// Превратить отрицательные значения в положительные.
/// Turn negative values into positive values.
__m128d factor = _mm_cmpge_pd(val, getZero());
factor = _mm_min_pd(factor, getTwo());
factor = _mm_sub_pd(factor, getOne());
val = _mm_mul_pd(val, factor);
/// Алгоритм округления.
/// Rounding algorithm.
val = _mm_div_pd(val, scale);
__m128d res = _mm_cmpge_pd(val, getOneTenth());
val = _mm_round_pd(val, rounding_mode);
val = _mm_mul_pd(val, scale);
val = _mm_and_pd(val, res);
/// Вернуть настоящие знаки всех значений.
/// Return the real signs of all values.
val = _mm_mul_pd(val, factor);
normalize(val, mask);
@ -538,7 +538,7 @@ public:
}
};
#else
/// Реализация для ARM. Не векторизована. Не исправляет отрицательные нули.
/// Implementation for ARM. Not vectorized. Does not fix negative zeros.
template <int mode>
float roundWithMode(float x)
@ -613,12 +613,12 @@ public:
#endif
/** Реализация высокоуровневых функций округления.
/** Implementing high-level rounding functions.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
struct FunctionRoundingImpl;
/** Реализация высокоуровневых функций округления для целочисленных значений.
/** Implement high-level rounding functions for integer values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
@ -650,7 +650,7 @@ public:
}
};
/** Реализация высокоуровневых функций округления для значений с плавающей точкой.
/** Implement high-level round-off functions for floating-point values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
@ -730,7 +730,7 @@ public:
}
};
/** Реализация высокоуровневых функций округления в том случае, когда возвращается нулевое значение.
/** Implementation of high-level rounding functions in the case when a zero value is returned.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
@ -748,11 +748,11 @@ public:
}
};
/// Следующий код генерирует во время сборки таблицу степеней числа 10.
/// The following code generates a table of powers of 10 during the build.
namespace
{
/// Отдельные степени числа 10.
/// Individual degrees of the number 10.
template<size_t N>
struct PowerOf10
@ -767,7 +767,7 @@ namespace
};
}
/// Объявление и определение контейнера содержащего таблицу степеней числа 10.
/// Declaring and defining a container containing a table of powers of 10.
template<size_t... TArgs>
struct TableContainer
@ -778,7 +778,7 @@ struct TableContainer
template<size_t... TArgs>
const std::array<size_t, sizeof...(TArgs)> TableContainer<TArgs...>::values {{ TArgs... }};
/// Генератор первых N степеней.
/// The generator of the first N degrees.
template<size_t N, size_t... TArgs>
struct FillArrayImpl
@ -798,9 +798,9 @@ struct FillArray
using result = typename FillArrayImpl<N - 1>::result;
};
/** Этот шаблон определяет точность, которую используют функции round/ceil/floor,
* затем преобразовывает её в значение, которое можно использовать в операциях
* умножения и деления. Поэтому оно называется масштабом.
/** This pattern defines the precision that the round/ceil/floor functions use,
* then converts it to a value that can be used in operations of
* multiplication and division. Therefore, it is called a scale.
*/
template<typename T, typename U, typename Enable = void>
struct ScaleForRightType;
@ -944,7 +944,7 @@ struct ScaleForRightType<T, U,
}
};
/** Превратить параметр точности в масштаб.
/** Turn the precision parameter into a scale.
*/
template<typename T>
struct ScaleForLeftType
@ -968,7 +968,7 @@ struct ScaleForLeftType
}
};
/** Главный шаблон применяющий функцию округления к значению или столбцу.
/** The main template that applies the rounding function to a value or column.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
struct Cruncher
@ -997,7 +997,7 @@ struct Cruncher
}
};
/** Выбрать подходящий алгоритм обработки в зависимости от масштаба.
/** Select the appropriate processing algorithm depending on the scale.
*/
template<typename T, template <typename> class U, int rounding_mode>
struct Dispatcher
@ -1028,9 +1028,9 @@ struct Dispatcher
}
};
/** Шаблон для функций, которые округляют значение входного параметра типа
* (U)Int8/16/32/64 или Float32/64, и принимают дополнительный необязятельный
* параметр (по умолчанию - 0).
/** A template for functions that round the value of an input parameter of type
* (U)Int8/16/32/64 or Float32/64, and accept an additional optional
* parameter (default is 0).
*/
template<typename Name, int rounding_mode>
class FunctionRounding : public IFunction
@ -1072,7 +1072,7 @@ public:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if ((arguments.size() < 1) || (arguments.size() > 2))

View File

@ -102,6 +102,7 @@ struct LengthImpl
/** If the string is UTF-8 encoded text, it returns the length of the text in code points.
* (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization)
* (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization)
* Otherwise, the behavior is undefined.
*/
struct LengthUTF8Impl

View File

@ -13,7 +13,7 @@
namespace DB
{
/** Функции работы со строками:
/** String functions
*
* length, empty, notEmpty,
* concat, substring, lower, upper, reverse
@ -26,11 +26,11 @@ namespace DB
* s, c1, c2 -> s: substring, substringUTF8
* s, c1, c2, s2 -> s: replace, replaceUTF8
*
* Функции поиска строк и регулярных выражений расположены отдельно.
* Функции работы с URL расположены отдельно.
* Функции кодирования строк, конвертации в другие типы расположены отдельно.
* The search functions for strings and regular expressions are located separately.
* URL functions are located separately.
* String encoding functions, converting to other types are located separately.
*
* Функции length, empty, notEmpty, reverse также работают с массивами.
* The functions length, empty, notEmpty, reverse also work with arrays.
*/
@ -89,10 +89,10 @@ inline void UTF8CyrillicToCase(const UInt8 *& src, const UInt8 * const src_end,
}
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
* Замечание: предполагается, что после перевода символа в другой регистр,
* длина его мультибайтовой последовательности в UTF-8 не меняется.
* Иначе - поведение не определено.
/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case.
* Note: It is assumed that after the character is converted to another case,
* the length of its multibyte sequence in UTF-8 does not change.
* Otherwise, the behavior is undefined.
*/
template <char not_case_lower_bound,
char not_case_upper_bound,

View File

@ -15,31 +15,31 @@
namespace DB
{
/** Функции, разделяющие строки на массив строк или наоборот.
/** Functions that split strings into an array of strings or vice versa.
*
* splitByChar(sep, s)
* splitByString(sep, s)
* splitByRegexp(regexp, s)
*
* extractAll(s, regexp) - выделить из строки подпоследовательности, соответствующие регекспу.
* - первый subpattern, если в regexp-е есть subpattern;
* - нулевой subpattern (сматчившуюся часть, иначе);
* - инача, пустой массив
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
* - first subpattern, if regexp has subpattern;
* - zero subpattern (the match part, otherwise);
* - otherwise, an empty array
*
* arrayStringConcat(arr)
* arrayStringConcat(arr, delimiter)
* - склеить массив строк в одну строку через разделитель.
* - join an array of strings into one string via a separator.
*
* alphaTokens(s) - выделить из строки подпоследовательности [a-zA-Z]+.
* alphaTokens(s) - select from the string subsequence `[a-zA-Z]+`.
*
* Функции работы с URL расположены отдельно.
* URL functions are located separately.
*/
using Pos = const char *;
/// Генераторы подстрок. Все они обладают общим интерфейсом.
/// Substring generators. All of them have a common interface.
class AlphaTokensImpl
{
@ -48,13 +48,13 @@ private:
Pos end;
public:
/// Получить имя фукнции.
/// Get the name of the function.
static constexpr auto name = "alphaTokens";
static String getName() { return name; }
static size_t getNumberOfArguments() { return 1; }
/// Проверить типы агрументов функции.
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
{
if (!typeid_cast<const DataTypeString *>(&*arguments[0]))
@ -62,23 +62,23 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// Инициализировать по аргументам функции.
/// Initialize by the function arguments.
void init(Block & block, const ColumnNumbers & arguments) {}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
@ -141,7 +141,7 @@ public:
sep = sep_str[0];
}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 1;
@ -204,20 +204,20 @@ public:
sep = col->getData();
}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of strings
size_t getStringsArgumentPosition()
{
return 1;
}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos)
@ -252,13 +252,13 @@ public:
static String getName() { return name; }
static size_t getNumberOfArguments() { return 2; }
/// Проверить типы агрументов функции.
/// Check the type of function arguments.
static void checkArguments( const DataTypes & arguments )
{
SplitByStringImpl::checkArguments(arguments);
}
/// Инициализировать по аргументам функции.
/// Initialize by the function arguments.
void init(Block & block, const ColumnNumbers & arguments)
{
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(block.safeGetByPosition(arguments[1]).column.get());
@ -274,20 +274,20 @@ public:
matches.resize(capture + 1);
}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos || pos > end)
@ -305,7 +305,7 @@ public:
}
};
/// Функция, принимающая строку, и возвращающая массив подстрок, создаваемый некоторым генератором.
/// A function that takes a string, and returns an array of substrings created by some generator.
template <typename Generator>
class FunctionTokens : public IFunction
{
@ -350,7 +350,7 @@ public:
const ColumnString::Offsets_t & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
res_strings_offsets.reserve(src_offsets.size() * 5); /// Константа 5 - наугад.
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr;
@ -411,7 +411,7 @@ public:
};
/// Склеивает массив строк в одну строку через разделитель.
/// Joins an array of strings into one string via a separator.
class FunctionArrayStringConcat : public IFunction
{
private:
@ -428,14 +428,14 @@ private:
if (!size)
return;
/// С небольшим запасом - как будто разделитель идёт и после последней строки массива.
/// With a small margin - as if the separator goes after the last string of the array.
dst_chars.resize(
src_chars.size()
+ delimiter_size * src_string_offsets.size() /// Разделители после каждой строки...
+ src_array_offsets.size() /// Нулевой байт после каждой склеенной строки
- src_string_offsets.size()); /// Бывший нулевой байт после каждой строки массива
+ delimiter_size * src_string_offsets.size() /// Separators after each string...
+ src_array_offsets.size() /// Zero byte after each joined string
- src_string_offsets.size()); /// The former zero byte after each string of the array
/// Будет столько строк, сколько было массивов.
/// There will be as many strings as there were arrays.
dst_string_offsets.resize(src_array_offsets.size());
ColumnArray::Offset_t current_src_array_offset = 0;
@ -443,10 +443,10 @@ private:
ColumnString::Offset_t current_dst_string_offset = 0;
/// Цикл по массивам строк.
/// Loop through the array of strings.
for (size_t i = 0; i < size; ++i)
{
/// Цикл по строкам внутри массива. /// NOTE Можно всё сделать за одно копирование, если разделитель имеет размер 1.
/// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1.
for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset)
{
size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1;

View File

@ -10,31 +10,31 @@
namespace DB
{
/** Функции поиска и замены в строках:
/** Search and replace functions in strings:
*
* position(haystack, needle) - обычный поиск подстроки в строке, возвращает позицию (в байтах) найденной подстроки, начиная с 1, или 0, если подстрока не найдена.
* positionUTF8(haystack, needle) - то же самое, но позиция вычисляется в кодовых точках, при условии, что строка в кодировке UTF-8.
* position(haystack, needle) - the normal search for a substring in a string, returns the position (in bytes) of the found substring starting with 1, or 0 if no substring is found.
* positionUTF8(haystack, needle) - the same, but the position is calculated at code points, provided that the string is encoded in UTF-8.
* positionCaseInsensitive(haystack, needle)
* positionCaseInsensitiveUTF8(haystack, needle)
*
* like(haystack, pattern) - поиск по регулярному выражению LIKE; возвращает 0 или 1. Регистронезависимое, но только для латиницы.
* like(haystack, pattern) - search by the regular expression LIKE; Returns 0 or 1. Case-insensitive, but only for Latin.
* notLike(haystack, pattern)
*
* match(haystack, pattern) - поиск по регулярному выражению re2; возвращает 0 или 1.
* match(haystack, pattern) - search by regular expression re2; Returns 0 or 1.
*
* Применяет регексп re2 и достаёт:
* - первый subpattern, если в regexp-е есть subpattern;
* - нулевой subpattern (сматчившуюся часть, иначе);
* - если не сматчилось - пустую строку.
* Applies regexp re2 and pulls:
* - the first subpattern, if the regexp has a subpattern;
* - the zero subpattern (the match part, otherwise);
* - if not match - an empty string.
* extract(haystack, pattern)
*
* replaceOne(haystack, pattern, replacement) - замена шаблона по заданным правилам, только первое вхождение.
* replaceAll(haystack, pattern, replacement) - замена шаблона по заданным правилам, все вхождения.
* replaceOne(haystack, pattern, replacement) - replacing the pattern with the specified rules, only the first occurrence.
* replaceAll(haystack, pattern, replacement) - replacing the pattern with the specified rules, all occurrences.
*
* replaceRegexpOne(haystack, pattern, replacement) - замена шаблона по заданному регекспу, только первое вхождение.
* replaceRegexpAll(haystack, pattern, replacement) - замена шаблона по заданному регекспу, все вхождения.
* replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence.
* replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences.
*
* Внимание! На данный момент, аргументы needle, pattern, n, replacement обязаны быть константами.
* Warning! At this point, the arguments needle, pattern, n, replacement must be constants.
*/

View File

@ -25,30 +25,30 @@ namespace ErrorCodes
}
/** transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
/** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match.
*/
DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeTraits::EnrichedDataTypePtr & type1, const IDataType & type2);
/** transform(x, [from...], [to...], default)
* - преобразует значения согласно явно указанному отображению.
* - converts the values according to the explicitly specified mapping.
*
* x - что преобразовывать.
* from - константный массив значений для преобразования.
* to - константный массив значений, в которые должны быть преобразованы значения из from.
* default - какое значение использовать, если x не равен ни одному из значений во from.
* from и to - массивы одинаковых размеров.
* x - what to transform.
* from - a constant array of values for the transformation.
* to - a constant array of values into which values from `from` must be transformed.
* default - what value to use if x is not equal to any of the values in `from`.
* `from` and `to` - arrays of the same size.
*
* Типы:
* Types:
* transform(T, Array(T), Array(U), U) -> U
*
* transform(x, [from...], [to...])
* - eсли default не указан, то для значений x, для которых нет соответствующего элемента во from, возвращается не изменённое значение x.
* - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned.
*
* Типы:
* Types:
* transform(T, Array(T), Array(T)) -> T
*
* Замечание: реализация довольно громоздкая.
* Note: the implementation is rather cumbersome.
*/
class FunctionTransform : public IFunction
{
@ -131,12 +131,12 @@ public:
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
/// We take the smallest common type for the elements of the array of values `to` and for `default`.
DataTypeTraits::EnrichedDataTypePtr res = getSmallestCommonNumericType(enriched_type_arr_to_nested, *type_default);
return res.first;
}
/// TODO Больше проверок.
/// TODO More checks.
return type_arr_to_nested->clone();
}
}
@ -189,7 +189,7 @@ public:
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
/// Construct a block of full-size columns of size 1 and compute the function as usual.
Block tmp_block;
ColumnNumbers tmp_arguments;
@ -724,10 +724,10 @@ private:
}
/// Разные варианты хэш-таблиц для реализации отображения.
/// Different versions of the hash tables to implement the mapping.
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
using NumToString = HashMap <UInt64, StringRef, HashCRC32 <UInt64 >>; /// Everywhere StringRef's with trailing zero.
using StringToNum = HashMap<StringRef, UInt64, StringRefHash>;
using StringToString = HashMap<StringRef, StringRef, StringRefHash>;
@ -738,12 +738,12 @@ private:
Arena string_pool;
Field const_default_value; /// Null, если не задано.
Field const_default_value; /// Null, if not specified.
bool prepared = false;
std::mutex mutex;
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
/// Can be called from different threads. It works only on the first call.
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
{
if (prepared)
@ -764,7 +764,7 @@ private:
Array converted_to;
const Array * used_to = &to;
/// Задано ли значение по-умолчанию.
/// Whether the default value is set.
if (arguments.size() == 4)
{
@ -774,7 +774,7 @@ private:
if (const_default_col)
const_default_value = (*const_default_col)[0];
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
/// Do I need to convert the elements `to` and `default_value` to the smallest common type that is Float64?
bool default_col_is_float =
typeid_cast<const ColumnFloat32 *>(default_col)
|| typeid_cast<const ColumnFloat64 *>(default_col)
@ -797,7 +797,7 @@ private:
}
}
/// Замечание: не делается проверка дубликатов в массиве from.
/// Note: Do not check the duplicates in the `from` array.
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
{

View File

@ -258,7 +258,7 @@ struct ExtractTopLevelDomain
if (!last_dot)
return;
/// Для IPv4-адресов не выделяем ничего.
/// For IPv4 addresses select nothing.
if (last_dot[1] <= '9')
return;
@ -380,7 +380,7 @@ struct ExtractQueryStringAndFragment
}
};
/// С точкой на конце.
/// With dot at the end.
struct ExtractWWW
{
static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size)
@ -526,11 +526,11 @@ struct CutURLParameterImpl
begin_pos = pos;
end_pos = begin_pos + param_len;
/// Пропустим значение.
/// Skip the value.
while (*end_pos && *end_pos != '&' && *end_pos != '#')
++end_pos;
/// Захватим '&' до или после параметра.
/// Capture '&' before or after the parameter.
if (*end_pos == '&')
++end_pos;
else if (begin_pos[-1] == '&')
@ -573,13 +573,13 @@ public:
void init(Block & block, const ColumnNumbers & arguments) {}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
{
return 0;
}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
@ -587,7 +587,7 @@ public:
first = true;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (pos == nullptr)
@ -656,7 +656,7 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
{
return 0;
@ -664,7 +664,7 @@ public:
void init(Block & block, const ColumnNumbers & arguments) {}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
@ -672,7 +672,7 @@ public:
first = true;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (pos == nullptr)
@ -735,38 +735,37 @@ public:
void init(Block & block, const ColumnNumbers & arguments) {}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
{
return 0;
}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
begin = pos = pos_;
end = end_;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Код из URLParser.
/// Code from URLParser.
if (pos == end)
return false;
if (pos == begin)
{
/// Распарсим всё, что идёт до пути
/// Let's parse everything that goes before the path
/// Предположим, что протокол уже переведён в нижний регистр.
/// Assume that the protocol has already been changed to lowercase.
while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9')))
++pos;
/** Будем вычислять иерархию только для URL-ов, в которых есть протокол, и после него идут два слеша.
* (http, file - подходят, mailto, magnet - не подходят), и после двух слешей ещё хоть что-нибудь есть
* Для остальных просто вернём полный URL как единственный элемент иерархии.
/** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes.
* (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there
* For the rest, simply return the full URL as the only element of the hierarchy.
*/
if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end))
{
@ -776,7 +775,7 @@ public:
return true;
}
/// Доменом для простоты будем считать всё, что после протокола и двух слешей, до следующего слеша или до ? или до #
/// The domain for simplicity is everything that after the protocol and two slashes, until the next slash or `?` or `#`
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
@ -789,7 +788,7 @@ public:
return true;
}
/// Идём до следующего / или ? или #, пропуская все те, что вначале.
/// We go to the next `/` or `?` or `#`, skipping all those at the beginning.
while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos == end)
@ -831,13 +830,13 @@ public:
void init(Block & block, const ColumnNumbers & arguments) {}
/// Возвращает позицию аргумента, являющегося столбцом строк
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
{
return 0;
}
/// Вызывается для каждой следующей строки.
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
begin = pos = pos_;
@ -845,25 +844,25 @@ public:
end = end_;
}
/// Получить следующий токен, если есть, или вернуть false.
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Код из URLParser.
/// Code from URLParser.
if (pos == end)
return false;
if (pos == begin)
{
/// Распарсим всё, что идёт до пути
/// Let's parse everything that goes before the path
/// Предположим, что протокол уже переведён в нижний регистр.
/// Assume that the protocol has already been changed to lowercase.
while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9')))
++pos;
/** Будем вычислять иерархию только для URL-ов, в которых есть протокол, и после него идут два слеша.
* (http, file - подходят, mailto, magnet - не подходят), и после двух слешей ещё хоть что-нибудь есть
* Для остальных просто вернём пустой массив.
/** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes.
* (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there.
* For the rest, just return an empty array.
*/
if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end))
{
@ -871,7 +870,7 @@ public:
return false;
}
/// Доменом для простоты будем считать всё, что после протокола и двух слешей, до следующего слеша или до ? или до #
/// The domain for simplicity is everything that after the protocol and the two slashes, until the next slash or `?` or `#`
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
@ -881,7 +880,7 @@ public:
++pos;
}
/// Идём до следующего / или ? или #, пропуская все те, что вначале.
/// We go to the next `/` or `?` or `#`, skipping all those at the beginning.
while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos == end)
@ -900,7 +899,7 @@ public:
};
/** Выделить кусок строки, используя Extractor.
/** Select part of string using the Extractor.
*/
template <typename Extractor>
struct ExtractSubstringImpl
@ -915,7 +914,7 @@ struct ExtractSubstringImpl
size_t prev_offset = 0;
size_t res_offset = 0;
/// Выделенный кусок.
/// Matched part.
Pos start;
size_t length;
@ -950,7 +949,7 @@ struct ExtractSubstringImpl
};
/** Удалить кусок строки, используя Extractor.
/** Delete part of string using the Extractor.
*/
template <typename Extractor>
struct CutSubstringImpl
@ -965,7 +964,7 @@ struct CutSubstringImpl
size_t prev_offset = 0;
size_t res_offset = 0;
/// Выделенный кусок.
/// Matched part.
Pos start;
size_t length;

Some files were not shown because too many files have changed in this diff Show More