Merge branch 'master' into database_atomic

This commit is contained in:
Alexander Tokmakov 2020-03-30 17:53:40 +03:00
commit 731aeb2dbe
681 changed files with 91278 additions and 2027 deletions

View File

@ -18,3 +18,4 @@ ClickHouse is an open-source column-oriented database management system that all
* [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date.
* [Talks on Saint HighLoad++ in St. Petersburg](https://www.highload.ru/spb/2020/abstracts/6647) on TBD date.
* [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date.
* [ClickHouse in Avito (online in Russian)](https://avitotech.timepad.ru/event/1290051/) on April 9, 2020.

View File

@ -367,137 +367,7 @@ void BaseDaemon::reloadConfiguration()
}
namespace
{
enum class InstructionFail
{
NONE = 0,
SSE3 = 1,
SSSE3 = 2,
SSE4_1 = 3,
SSE4_2 = 4,
AVX = 5,
AVX2 = 6,
AVX512 = 7
};
std::string instructionFailToString(InstructionFail fail)
{
switch (fail)
{
case InstructionFail::NONE:
return "NONE";
case InstructionFail::SSE3:
return "SSE3";
case InstructionFail::SSSE3:
return "SSSE3";
case InstructionFail::SSE4_1:
return "SSE4.1";
case InstructionFail::SSE4_2:
return "SSE4.2";
case InstructionFail::AVX:
return "AVX";
case InstructionFail::AVX2:
return "AVX2";
case InstructionFail::AVX512:
return "AVX512";
}
__builtin_unreachable();
}
sigjmp_buf jmpbuf;
void sigIllCheckHandler(int, siginfo_t *, void *)
{
siglongjmp(jmpbuf, 1);
}
/// Check if necessary sse extensions are available by trying to execute some sse instructions.
/// If instruction is unavailable, SIGILL will be sent by kernel.
void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
{
#if __SSE3__
fail = InstructionFail::SSE3;
__asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSSE3__
fail = InstructionFail::SSSE3;
__asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSE4_1__
fail = InstructionFail::SSE4_1;
__asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSE4_2__
fail = InstructionFail::SSE4_2;
__asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __AVX__
fail = InstructionFail::AVX;
__asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX2__
fail = InstructionFail::AVX2;
__asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX512__
fail = InstructionFail::AVX512;
__asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0");
#endif
fail = InstructionFail::NONE;
}
/// Check SSE and others instructions availability
/// Calls exit on fail
void checkRequiredInstructions()
{
struct sigaction sa{};
struct sigaction sa_old{};
sa.sa_sigaction = sigIllCheckHandler;
sa.sa_flags = SA_SIGINFO;
auto signal = SIGILL;
if (sigemptyset(&sa.sa_mask) != 0
|| sigaddset(&sa.sa_mask, signal) != 0
|| sigaction(signal, &sa, &sa_old) != 0)
{
std::cerr << "Can not set signal handler\n";
exit(1);
}
volatile InstructionFail fail = InstructionFail::NONE;
if (sigsetjmp(jmpbuf, 1))
{
std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n";
exit(1);
}
checkRequiredInstructionsImpl(fail);
if (sigaction(signal, &sa_old, nullptr))
{
std::cerr << "Can not set signal handler\n";
exit(1);
}
}
}
BaseDaemon::BaseDaemon()
{
checkRequiredInstructions();
}
BaseDaemon::BaseDaemon() = default;
BaseDaemon::~BaseDaemon()

View File

@ -3,19 +3,21 @@
#include <Interpreters/Cluster.h>
namespace DB
{
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
using DatabaseAndTableName = std::pair<String, String>;
using ListOfDatabasesAndTableNames = std::vector<DatabaseAndTableName>;
/// Hierarchical description of the tasks
struct ShardPartitionPiece;
struct ShardPartition;
struct TaskShard;
struct TaskTable;
struct TaskCluster;
struct ClusterPartition;
using PartitionPieces = std::vector<ShardPartitionPiece>;
using TasksPartition = std::map<String, ShardPartition, std::greater<>>;
using ShardInfo = Cluster::ShardInfo;
using TaskShardPtr = std::shared_ptr<TaskShard>;

View File

@ -12,6 +12,6 @@ set(CLICKHOUSE_COPIER_LINK PRIVATE
clickhouse_dictionaries
string_utils ${Poco_XML_LIBRARY} PUBLIC daemon)
set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR})
set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR} ${CMAKE_CURRENT_SOURCE_DIR})
clickhouse_program_add(copier)

File diff suppressed because it is too large Load Diff

View File

@ -5,8 +5,10 @@
#include "TaskCluster.h"
#include "TaskTableAndShard.h"
#include "ShardPartition.h"
#include "ShardPartitionPiece.h"
#include "ZooKeeperStaff.h"
namespace DB
{
@ -27,10 +29,10 @@ public:
void init();
template<typename T>
template <typename T>
decltype(auto) retry(T && func, UInt64 max_tries = 100);
void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) ;
void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard);
/// Compute set of partitions, assume set of partitions aren't changed during the processing
void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0);
@ -54,6 +56,10 @@ public:
copy_fault_probability = copy_fault_probability_;
}
void setMoveFaultProbability(double move_fault_probability_)
{
move_fault_probability = move_fault_probability_;
}
protected:
@ -73,59 +79,102 @@ protected:
}
zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed(
const zkutil::ZooKeeperPtr &zookeeper,
const String &description,
const zkutil::ZooKeeperPtr & zookeeper,
const String & description,
bool unprioritized);
/*
* Checks that partition piece or some other entity is clean.
* The only requirement is that you have to pass is_dirty_flag_path and is_dirty_cleaned_path to the function.
* And is_dirty_flag_path is a parent of is_dirty_cleaned_path.
* */
static bool checkPartitionPieceIsClean(
const zkutil::ZooKeeperPtr & zookeeper,
const CleanStateClock & clean_state_clock,
const String & task_status_path);
bool checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition);
/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock.
* State of some task could change during the processing.
* We have to ensure that all shards have the finished state and there is no dirty flag.
* Moreover, we have to check status twice and check zxid, because state can change during the checking.
*/
bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name,
const TasksShard & shards_with_partition);
/* The same as function above
* Assume that we don't know on which shards do we have partition certain piece.
* We'll check them all (I mean shards that contain the whole partition)
* And shards that don't have certain piece MUST mark that piece is_done true.
* */
bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name,
size_t piece_number, const TasksShard & shards_with_partition);
/*Alter successful insertion to helping tables it will move all pieces to destination table*/
TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name);
/// Removes MATERIALIZED and ALIAS columns from create table query
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr &query_ast);
bool tryDropPartition(ShardPartition & task_partition,
const zkutil::ZooKeeperPtr & zookeeper,
const CleanStateClock & clean_state_clock);
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast);
bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number,
const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock);
static constexpr UInt64 max_table_tries = 1000;
static constexpr UInt64 max_shard_partition_tries = 600;
static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 100;
bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table);
PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts,
ShardPartition & task_partition,
bool is_unprioritized_task);
/// Job for copying partition from particular shard.
TaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts,
ShardPartition & task_partition,
bool is_unprioritized_task);
PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts,
ShardPartition & task_partition,
bool is_unprioritized_task);
TaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts,
ShardPartition & task_partition,
bool is_unprioritized_task);
TaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts,
ShardPartition & task_partition,
const size_t current_piece_number,
bool is_unprioritized_task);
void dropAndCreateLocalTable(const ASTPtr & create_ast);
void dropLocalTableIfExists (const DatabaseAndTableName & table_name) const;
void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const;
String getRemoteCreateTable(const DatabaseAndTableName & table,
Connection & connection,
const Settings * settings = nullptr);
void dropHelpingTables(const TaskTable & task_table);
/// Is used for usage less disk space.
/// After all pieces were successfully moved to original destination
/// table we can get rid of partition pieces (partitions in helping tables).
void dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name);
String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr);
ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard);
void createShardInternalTables(const ConnectionTimeouts & timeouts,
TaskShard & task_shard,
bool create_split = true);
/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it.
void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true);
std::set<String> getShardPartitions(const ConnectionTimeouts & timeouts,
TaskShard & task_shard);
std::set<String> getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard);
bool checkShardHasPartition(const ConnectionTimeouts & timeouts,
TaskShard & task_shard,
const String & partition_quoted_name);
bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name);
bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts,
TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number);
/*
* This class is used in executeQueryOnCluster function
* You can execute query on each shard (no sense it is executed on each replica of a shard or not)
* or you can execute query on each replica on each shard.
* First mode is useful for INSERTS queries.
* */
enum ClusterExecutionMode
{
ON_EACH_SHARD,
ON_EACH_NODE
};
/** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster
* Returns number of shards for which at least one replica executed query successfully
@ -136,6 +185,7 @@ protected:
const ASTPtr & query_ast_ = nullptr,
const Settings * settings = nullptr,
PoolMode pool_mode = PoolMode::GET_ALL,
ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD,
UInt64 max_successful_executions_per_shard = 0) const;
private:
@ -145,25 +195,25 @@ private:
String working_database_name;
/// Auto update config stuff
UInt64 task_descprtion_current_version = 1;
std::atomic<UInt64> task_descprtion_version{1};
UInt64 task_description_current_version = 1;
std::atomic<UInt64> task_description_version{1};
Coordination::WatchCallback task_description_watch_callback;
/// ZooKeeper session used to set the callback
zkutil::ZooKeeperPtr task_description_watch_zookeeper;
ConfigurationPtr task_cluster_initial_config;
ConfigurationPtr task_cluster_current_config;
Coordination::Stat task_descprtion_current_stat{};
Coordination::Stat task_description_current_stat{};
std::unique_ptr<TaskCluster> task_cluster;
bool is_safe_mode = false;
double copy_fault_probability = 0.0;
double move_fault_probability = 0.0;
Context & context;
Poco::Logger * log;
std::chrono::milliseconds default_sleep_time{1000};
};
}

View File

@ -17,6 +17,8 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
is_safe_mode = config().has("safe-mode");
if (config().has("copy-fault-probability"))
copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0);
if (config().has("move-fault-probability"))
move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0);
base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current();
// process_id is '<hostname>#<start_timestamp>_<pid>'
time_t timestamp = Poco::Timestamp().epochTime();
@ -58,23 +60,25 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
Base::defineOptions(options);
options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper")
.argument("task-path").binding("task-path"));
.argument("task-path").binding("task-path"));
options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path")
.argument("task-file").binding("task-file"));
.argument("task-file").binding("task-file"));
options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists")
.argument("task-upload-force").binding("task-upload-force"));
.argument("task-upload-force").binding("task-upload-force"));
options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors")
.binding("safe-mode"));
.binding("safe-mode"));
options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)")
.argument("copy-fault-probability").binding("copy-fault-probability"));
.argument("copy-fault-probability").binding("copy-fault-probability"));
options.addOption(Poco::Util::Option("move-fault-probability", "", "the moving fails with specified probability (used to test partition state recovering)")
.argument("move-fault-probability").binding("move-fault-probability"));
options.addOption(Poco::Util::Option("log-level", "", "sets log level")
.argument("log-level").binding("log-level"));
.argument("log-level").binding("log-level"));
options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories")
.argument("base-dir").binding("base-dir"));
.argument("base-dir").binding("base-dir"));
using Me = std::decay_t<decltype(*this)>;
options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help")
.callback(Poco::Util::OptionCallback<Me>(this, &Me::handleHelp)));
.callback(Poco::Util::OptionCallback<Me>(this, &Me::handleHelp)));
}
@ -85,10 +89,10 @@ void ClusterCopierApp::mainImpl()
auto log = &logger();
LOG_INFO(log, "Starting clickhouse-copier ("
<< "id " << process_id << ", "
<< "host_id " << host_id << ", "
<< "path " << process_path << ", "
<< "revision " << ClickHouseRevision::get() << ")");
<< "id " << process_id << ", "
<< "host_id " << host_id << ", "
<< "path " << process_path << ", "
<< "revision " << ClickHouseRevision::get() << ")");
auto context = std::make_unique<Context>(Context::createGlobal());
context->makeGlobalContext();
@ -115,6 +119,7 @@ void ClusterCopierApp::mainImpl()
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, *context);
copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability);
copier->setMoveFaultProbability(move_fault_probability);
auto task_file = config().getString("task-file", "");
if (!task_file.empty())

View File

@ -78,7 +78,8 @@ private:
std::string task_path;
std::string log_level = "trace";
bool is_safe_mode = false;
double copy_fault_probability = 0;
double copy_fault_probability = 0.0;
double move_fault_probability = 0.0;
bool is_help = false;
std::string base_dir;

View File

@ -1,4 +1,5 @@
#include "Internals.h"
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
@ -7,6 +8,8 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data)
{
std::stringstream ss(xml_data);
@ -14,7 +17,6 @@ ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data)
return {new Poco::Util::XMLConfiguration{&input_source}};
}
String getQuotedTable(const String & database, const String & table)
{
if (database.empty())
@ -114,6 +116,142 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
}
}
ASTPtr extractPrimaryKey(const ASTPtr & storage_ast)
{
String storage_str = queryToString(storage_ast);
const auto & storage = storage_ast->as<ASTStorage &>();
const auto & engine = storage.engine->as<ASTFunction &>();
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
}
if (!isExtendedDefinitionStorage(storage_ast))
{
throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.",
ErrorCodes::BAD_ARGUMENTS);
}
if (storage.primary_key)
return storage.primary_key->clone();
return nullptr;
}
ASTPtr extractOrderBy(const ASTPtr & storage_ast)
{
String storage_str = queryToString(storage_ast);
const auto & storage = storage_ast->as<ASTStorage &>();
const auto & engine = storage.engine->as<ASTFunction &>();
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
}
if (!isExtendedDefinitionStorage(storage_ast))
{
throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.",
ErrorCodes::BAD_ARGUMENTS);
}
if (storage.order_by)
return storage.order_by->clone();
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
}
String createCommaSeparatedStringFrom(const Names & names)
{
std::ostringstream ss;
if (!names.empty())
{
std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator<std::string>(ss, ", "));
ss << names.back();
}
return ss.str();
}
Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast)
{
const auto sorting_key_ast = extractOrderBy(storage_ast);
const auto primary_key_ast = extractPrimaryKey(storage_ast);
const auto sorting_key_expr_list = MergeTreeData::extractKeyExpressionList(sorting_key_ast);
const auto primary_key_expr_list = primary_key_ast
? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone();
/// Maybe we have to handle VersionedCollapsing engine separately. But in our case in looks pointless.
size_t primary_key_size = primary_key_expr_list->children.size();
size_t sorting_key_size = sorting_key_expr_list->children.size();
if (primary_key_size > sorting_key_size)
throw Exception("Primary key must be a prefix of the sorting key, but its length: "
+ toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size),
ErrorCodes::BAD_ARGUMENTS);
Names primary_key_columns;
Names sorting_key_columns;
NameSet primary_key_columns_set;
for (size_t i = 0; i < sorting_key_size; ++i)
{
String sorting_key_column = sorting_key_expr_list->children[i]->getColumnName();
sorting_key_columns.push_back(sorting_key_column);
if (i < primary_key_size)
{
String pk_column = primary_key_expr_list->children[i]->getColumnName();
if (pk_column != sorting_key_column)
throw Exception("Primary key must be a prefix of the sorting key, but in position "
+ toString(i) + " its column is " + pk_column + ", not " + sorting_key_column,
ErrorCodes::BAD_ARGUMENTS);
if (!primary_key_columns_set.emplace(pk_column).second)
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
primary_key_columns.push_back(pk_column);
}
}
return primary_key_columns;
}
String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast)
{
String storage_str = queryToString(storage_ast);
const auto & storage = storage_ast->as<ASTStorage &>();
const auto & engine = storage.engine->as<ASTFunction &>();
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception(
"Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
}
if (!startsWith(engine.name, "Replicated"))
{
return "";
}
auto replicated_table_arguments = engine.arguments->children;
auto zk_table_path_ast = replicated_table_arguments[0]->as<ASTLiteral &>();
auto zk_table_path_string = zk_table_path_ast.value.safeGet<String>();
return zk_table_path_string;
}
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random)
{
ShardPriority res;

View File

@ -31,6 +31,7 @@
#include <Common/ThreadStatus.h>
#include <Client/Connection.h>
#include <Interpreters/Context.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/InterpreterExistsQuery.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
@ -144,15 +145,15 @@ struct ShardPriority
}
};
/// Execution status of a task
enum class PartitionTaskStatus
/// Execution status of a task.
/// Is used for: partition copying task status, partition piece copying task status, partition moving task status.
enum class TaskStatus
{
Active,
Finished,
Error,
};
struct MultiTransactionInfo
{
int32_t code;
@ -174,6 +175,35 @@ bool isExtendedDefinitionStorage(const ASTPtr & storage_ast);
ASTPtr extractPartitionKey(const ASTPtr & storage_ast);
/*
* Choosing a Primary Key that Differs from the Sorting Key
* It is possible to specify a primary key (an expression with values that are written in the index file for each mark)
* that is different from the sorting key (an expression for sorting the rows in data parts).
* In this case the primary key expression tuple must be a prefix of the sorting key expression tuple.
* This feature is helpful when using the SummingMergeTree and AggregatingMergeTree table engines.
* In a common case when using these engines, the table has two types of columns: dimensions and measures.
* Typical queries aggregate values of measure columns with arbitrary GROUP BY and filtering by dimensions.
* Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key,
* it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns
* and this list must be frequently updated with newly added dimensions.
* In this case it makes sense to leave only a few columns in the primary key that will provide efficient
* range scans and add the remaining dimension columns to the sorting key tuple.
* ALTER of the sorting key is a lightweight operation because when a new column is simultaneously added t
* o the table and to the sorting key, existing data parts don't need to be changed.
* Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column,
* the data is sorted by both the old and new sorting keys at the moment of table modification.
*
* */
ASTPtr extractPrimaryKey(const ASTPtr & storage_ast);
ASTPtr extractOrderBy(const ASTPtr & storage_ast);
String createCommaSeparatedStringFrom(const Names & names);
Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast);
String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast);
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random);
}

View File

@ -1,24 +1,46 @@
#pragma once
#include "Aliases.h"
#include "TaskTableAndShard.h"
namespace DB
{
/// Just destination partition of a shard
/// I don't know what this comment means.
/// In short, when we discovered what shards contain currently processing partition,
/// This class describes a partition (name) that is stored on the shard (parent).
struct ShardPartition
{
ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {}
ShardPartition(TaskShard &parent, String name_quoted_, size_t number_of_splits = 10)
: task_shard(parent), name(std::move(name_quoted_)) { pieces.reserve(number_of_splits); }
String getPartitionPath() const;
String getPartitionPiecePath(size_t current_piece_number) const;
String getPartitionCleanStartPath() const;
String getPartitionPieceCleanStartPath(size_t current_piece_number) const;
String getCommonPartitionIsDirtyPath() const;
String getCommonPartitionIsCleanedPath() const;
String getPartitionActiveWorkersPath() const;
String getActiveWorkerPath() const;
String getPartitionShardsPath() const;
String getShardStatusPath() const;
/// What partition pieces are present in current shard.
/// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously)
/// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const;
/// Absent pieces have field is_absent_piece equals to true.
PartitionPieces pieces;
TaskShard & task_shard;
String name;
};
@ -28,11 +50,23 @@ inline String ShardPartition::getPartitionCleanStartPath() const
return getPartitionPath() + "/clean_start";
}
inline String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const
{
assert(current_piece_number < task_shard.task_table.number_of_splits);
return getPartitionPiecePath(current_piece_number) + "/clean_start";
}
inline String ShardPartition::getPartitionPath() const
{
return task_shard.task_table.getPartitionPath(name);
}
inline String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const
{
assert(current_piece_number < task_shard.task_table.number_of_splits);
return task_shard.task_table.getPartitionPiecePath(name, current_piece_number);
}
inline String ShardPartition::getShardStatusPath() const
{
// schema: /<root...>/tables/<table>/<partition>/shards/<shard>

View File

@ -0,0 +1,88 @@
#pragma once
#include "Internals.h"
namespace DB
{
struct ShardPartitionPiece
{
ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_present_piece_)
: is_absent_piece(!is_present_piece_), current_piece_number(current_piece_number_),
shard_partition(parent) {}
String getPartitionPiecePath() const;
String getPartitionPieceCleanStartPath() const;
String getPartitionPieceIsDirtyPath() const;
String getPartitionPieceIsCleanedPath() const;
String getPartitionPieceActiveWorkersPath() const;
String getActiveWorkerPath() const ;
/// On what shards do we have current partition.
String getPartitionPieceShardsPath() const;
String getShardStatusPath() const;
String getPartitionPieceCleanerPath() const;
bool is_absent_piece;
const size_t current_piece_number;
ShardPartition & shard_partition;
};
inline String ShardPartitionPiece::getPartitionPiecePath() const
{
return shard_partition.getPartitionPath() + "/piece_" + toString(current_piece_number);
}
inline String ShardPartitionPiece::getPartitionPieceCleanStartPath() const
{
return getPartitionPiecePath() + "/clean_start";
}
inline String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const
{
return getPartitionPiecePath() + "/is_dirty";
}
inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const
{
return getPartitionPieceIsDirtyPath() + "/cleaned";
}
inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const
{
return getPartitionPiecePath() + "/partition_piece_active_workers";
}
inline String ShardPartitionPiece::getActiveWorkerPath() const
{
return getPartitionPieceActiveWorkersPath() + "/" + toString(shard_partition.task_shard.numberInCluster());
}
/// On what shards do we have current partition.
inline String ShardPartitionPiece::getPartitionPieceShardsPath() const
{
return getPartitionPiecePath() + "/shards";
}
inline String ShardPartitionPiece::getShardStatusPath() const
{
return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster());
}
inline String ShardPartitionPiece::getPartitionPieceCleanerPath() const
{
return getPartitionPieceIsDirtyPath() + "/cleaner";
}
}

View File

@ -21,16 +21,44 @@ struct TaskTable
TaskCluster & task_cluster;
/// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone()
/// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc.
String getPartitionPath(const String & partition_name) const;
String getPartitionIsDirtyPath(const String & partition_name) const;
String getPartitionIsCleanedPath(const String & partition_name) const;
String getPartitionTaskStatusPath(const String & partition_name) const;
String getPartitionAttachIsActivePath(const String & partition_name) const;
String getPartitionAttachIsDonePath(const String & partition_name) const;
String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionIsDirtyPath(const String & partition_name) const;
String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionIsCleanedPath(const String & partition_name) const;
String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionTaskStatusPath(const String & partition_name) const;
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const;
bool isReplicatedTable() const { return engine_push_zk_path != ""; }
/// Partitions will be splitted into number-of-splits pieces.
/// Each piece will be copied independently. (10 by default)
size_t number_of_splits;
String name_in_config;
/// Used as task ID
String table_id;
/// Column names in primary key
String primary_key_comma_separated;
/// Source cluster and table
String cluster_pull_name;
DatabaseAndTableName table_pull;
@ -40,14 +68,31 @@ struct TaskTable
DatabaseAndTableName table_push;
/// Storage of destination table
/// (tables that are stored on each shard of target cluster)
String engine_push_str;
ASTPtr engine_push_ast;
ASTPtr engine_push_partition_key_ast;
/// A Distributed table definition used to split data
/// First argument of Replicated...MergeTree()
String engine_push_zk_path;
ASTPtr rewriteReplicatedCreateQueryToPlain();
/*
* A Distributed table definition used to split data
* Distributed table will be created on each shard of default
* cluster to perform data copying and resharding
* */
String sharding_key_str;
ASTPtr sharding_key_ast;
ASTPtr engine_split_ast;
ASTPtr main_engine_split_ast;
/*
* To copy partiton piece form one cluster to another we have to use Distributed table.
* In case of usage separate table (engine_push) for each partiton piece,
* we have to use many Distributed tables.
* */
ASTs auxiliary_engine_split_asts;
/// Additional WHERE expression to filter input data
String where_condition_str;
@ -62,10 +107,15 @@ struct TaskTable
Strings enabled_partitions;
NameSet enabled_partitions_set;
/// Prioritized list of shards
/**
* Prioritized list of shards
* all_shards contains information about all shards in the table.
* So we have to check whether particular shard have current partiton or not while processing.
*/
TasksShard all_shards;
TasksShard local_shards;
/// All partitions of the current table.
ClusterPartitions cluster_partitions;
NameSet finished_cluster_partitions;
@ -76,7 +126,8 @@ struct TaskTable
{
auto it = cluster_partitions.find(partition_name);
if (it == cluster_partitions.end())
throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR);
throw Exception("There are no cluster partition " + partition_name + " in " + table_id,
ErrorCodes::LOGICAL_ERROR);
return it->second;
}
@ -85,13 +136,16 @@ struct TaskTable
UInt64 rows_copied = 0;
template <typename RandomEngine>
void initShards(RandomEngine && random_engine);
void initShards(RandomEngine &&random_engine);
};
struct TaskShard
{
TaskShard(TaskTable &parent, const ShardInfo &info_) : task_table(parent), info(info_) {}
TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_)
{
list_of_split_tables_on_shard.assign(task_table.number_of_splits, DatabaseAndTableName());
}
TaskTable & task_table;
@ -123,7 +177,8 @@ struct TaskShard
/// Internal distributed tables
DatabaseAndTableName table_read_shard;
DatabaseAndTableName table_split_shard;
DatabaseAndTableName main_table_split_shard;
ListOfDatabasesAndTableNames list_of_split_tables_on_shard;
};
@ -134,29 +189,62 @@ inline String TaskTable::getPartitionPath(const String & partition_name) const
+ "/" + escapeForFileName(partition_name); // 201701
}
inline String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const
inline String TaskTable::getPartitionAttachIsActivePath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/attach_active";
}
inline String TaskTable::getPartitionAttachIsDonePath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/attach_is_done";
}
inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const
{
assert(piece_number < number_of_splits);
return getPartitionPath(partition_name) + "/piece_" + toString(piece_number); // 1...number_of_splits
}
inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const
{
return getPartitionPath(partition_name) + "/is_dirty";
}
inline String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const
inline String TaskTable::getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const
{
return getPartitionIsDirtyPath(partition_name) + "/cleaned";
return getPartitionPiecePath(partition_name, piece_number) + "/is_dirty";
}
inline String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const
inline String TaskTable::getCertainPartitionIsCleanedPath(const String & partition_name) const
{
return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned";
}
inline String TaskTable::getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const
{
return getCertainPartitionPieceIsDirtyPath(partition_name, piece_number) + "/cleaned";
}
inline String TaskTable::getCertainPartitionTaskStatusPath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/shards";
}
inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_,
const String & table_key)
inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const
{
return getPartitionPiecePath(partition_name, piece_number) + "/shards";
}
inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config,
const String & prefix_, const String & table_key)
: task_cluster(parent)
{
String table_prefix = prefix_ + "." + table_key + ".";
name_in_config = table_key;
number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10);
cluster_pull_name = config.getString(table_prefix + "cluster_pull");
cluster_push_name = config.getString(table_prefix + "cluster_push");
@ -176,13 +264,27 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
ParserStorage parser_storage;
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0);
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast));
engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast);
}
sharding_key_str = config.getString(table_prefix + "sharding_key");
auxiliary_engine_split_asts.reserve(number_of_splits);
{
ParserExpressionWithOptionalAlias parser_expression(false);
sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0);
engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast);
main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second,
sharding_key_ast);
for (const auto piece_number : ext::range(0, number_of_splits))
{
auxiliary_engine_split_asts.emplace_back
(
createASTStorageDistributed(cluster_push_name, table_push.first,
table_push.second + "_piece_" + toString(piece_number), sharding_key_ast)
);
}
}
where_condition_str = config.getString(table_prefix + "where_condition", "");
@ -213,7 +315,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
else
{
/// Parse sequence of <partition>...</partition>
for (const String & key : keys)
for (const String &key : keys)
{
if (!startsWith(key, "partition"))
throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
@ -259,6 +361,25 @@ inline void TaskTable::initShards(RandomEngine && random_engine)
local_shards.assign(all_shards.begin(), it_first_remote);
}
inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
{
ASTPtr prev_engine_push_ast = engine_push_ast->clone();
auto & new_storage_ast = prev_engine_push_ast->as<ASTStorage &>();
auto & new_engine_ast = new_storage_ast.engine->as<ASTFunction &>();
auto & replicated_table_arguments = new_engine_ast.arguments->children;
/// Delete first two arguments of Replicated...MergeTree() table.
replicated_table_arguments.erase(replicated_table_arguments.begin());
replicated_table_arguments.erase(replicated_table_arguments.begin());
/// Remove replicated from name
new_engine_ast.name = new_engine_ast.name.substr(10);
return new_storage_ast.clone();
}
inline String DB::TaskShard::getDescription() const
{
@ -272,7 +393,7 @@ inline String DB::TaskShard::getDescription() const
inline String DB::TaskShard::getHostNameExample() const
{
auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
return replicas.at(0).readableString();
}

View File

@ -157,11 +157,8 @@ public:
bool is_clean() const
{
return
!is_stale()
&& (
!discovery_zxid.hasHappened()
|| (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid));
return !is_stale()
&& (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid));
}
bool is_stale() const

View File

@ -148,7 +148,6 @@ try
return Application::EXIT_OK;
}
context = std::make_unique<Context>(Context::createGlobal());
context->makeGlobalContext();
context->setApplicationType(Context::ApplicationType::LOCAL);
@ -180,7 +179,7 @@ try
setupUsers();
/// Limit on total number of concurrently executing queries.
/// There is no need for concurrent threads, override max_concurrent_queries.
/// There is no need for concurrent queries, override max_concurrent_queries.
context->getProcessList().setMaxSize(0);
/// Size of cache for uncompressed blocks. Zero means disabled.

View File

@ -1,3 +1,7 @@
#include <signal.h>
#include <setjmp.h>
#include <unistd.h>
#include <new>
#include <iostream>
#include <vector>
@ -118,6 +122,159 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name));
}
enum class InstructionFail
{
NONE = 0,
SSE3 = 1,
SSSE3 = 2,
SSE4_1 = 3,
SSE4_2 = 4,
AVX = 5,
AVX2 = 6,
AVX512 = 7
};
const char * instructionFailToString(InstructionFail fail)
{
switch (fail)
{
case InstructionFail::NONE:
return "NONE";
case InstructionFail::SSE3:
return "SSE3";
case InstructionFail::SSSE3:
return "SSSE3";
case InstructionFail::SSE4_1:
return "SSE4.1";
case InstructionFail::SSE4_2:
return "SSE4.2";
case InstructionFail::AVX:
return "AVX";
case InstructionFail::AVX2:
return "AVX2";
case InstructionFail::AVX512:
return "AVX512";
}
__builtin_unreachable();
}
sigjmp_buf jmpbuf;
[[noreturn]] void sigIllCheckHandler(int, siginfo_t *, void *)
{
siglongjmp(jmpbuf, 1);
}
/// Check if necessary SSE extensions are available by trying to execute some sse instructions.
/// If instruction is unavailable, SIGILL will be sent by kernel.
void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
{
#if defined(__SSE3__)
fail = InstructionFail::SSE3;
__asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if defined(__SSSE3__)
fail = InstructionFail::SSSE3;
__asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if defined(__SSE4_1__)
fail = InstructionFail::SSE4_1;
__asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if defined(__SSE4_2__)
fail = InstructionFail::SSE4_2;
__asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if defined(__AVX__)
fail = InstructionFail::AVX;
__asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if defined(__AVX2__)
fail = InstructionFail::AVX2;
__asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if defined(__AVX512__)
fail = InstructionFail::AVX512;
__asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0");
#endif
fail = InstructionFail::NONE;
}
/// This function is safe to use in static initializers.
void writeError(const char * data, size_t size)
{
while (size != 0)
{
ssize_t res = ::write(STDERR_FILENO, data, size);
if ((-1 == res || 0 == res) && errno != EINTR)
_Exit(1);
if (res > 0)
{
data += res;
size -= res;
}
}
}
/// Check SSE and others instructions availability. Calls exit on fail.
/// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions.
void checkRequiredInstructions()
{
struct sigaction sa{};
struct sigaction sa_old{};
sa.sa_sigaction = sigIllCheckHandler;
sa.sa_flags = SA_SIGINFO;
auto signal = SIGILL;
if (sigemptyset(&sa.sa_mask) != 0
|| sigaddset(&sa.sa_mask, signal) != 0
|| sigaction(signal, &sa, &sa_old) != 0)
{
/// You may wonder about strlen.
/// Typical implementation of strlen is using SSE4.2 or AVX2.
/// But this is not the case because it's compiler builtin and is executed at compile time.
const char * msg = "Can not set signal handler\n";
writeError(msg, strlen(msg));
_Exit(1);
}
volatile InstructionFail fail = InstructionFail::NONE;
if (sigsetjmp(jmpbuf, 1))
{
const char * msg1 = "Instruction check fail. The CPU does not support ";
writeError(msg1, strlen(msg1));
const char * msg2 = instructionFailToString(fail);
writeError(msg2, strlen(msg2));
const char * msg3 = " instruction set.\n";
writeError(msg3, strlen(msg3));
_Exit(1);
}
checkRequiredInstructionsImpl(fail);
if (sigaction(signal, &sa_old, nullptr))
{
const char * msg = "Can not set signal handler\n";
writeError(msg, strlen(msg));
_Exit(1);
}
}
struct Checker { Checker() { checkRequiredInstructions(); } } checker;
}

View File

@ -249,6 +249,10 @@ struct Settings : public SettingsCollection<Settings>
M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \
M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \
\
M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \
M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \
M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \
\
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \

View File

@ -352,6 +352,7 @@ private:
StringRef name;
StringRef description;
StringRef type;
bool is_important;
IsChangedFunction is_changed;
GetStringFunction get_string;
@ -398,6 +399,7 @@ public:
const_reference(const const_reference & src) = default;
const StringRef & getName() const { return member->name; }
const StringRef & getDescription() const { return member->description; }
const StringRef & getType() const { return member->type; }
bool isChanged() const { return member->is_changed(*collection); }
Field getValue() const;
String getValueAsString() const { return member->get_string(*collection); }

View File

@ -323,7 +323,9 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
add({StringRef(#NAME, strlen(#NAME)), \
StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
StringRef(#TYPE, strlen(#TYPE)), \
FLAGS & IMPORTANT, \
[](const Derived & d) { return d.NAME.changed; }, \
&Functions::NAME##_getString, &Functions::NAME##_getField, \

View File

@ -25,7 +25,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
* but it's clear that here is not the best place for this functionality.
*/
addTableLock(storage->lockStructureForShare(true, context.getInitialQueryId()));
addTableLock(storage->lockStructureForShare(context.getInitialQueryId()));
/// If the "root" table deduplactes blocks, there are no need to make deduplication for children
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
@ -54,7 +54,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
if (auto * materialized_view = dynamic_cast<StorageMaterializedView *>(dependent_table.get()))
{
addTableLock(materialized_view->lockStructureForShare(true, context.getInitialQueryId()));
addTableLock(materialized_view->lockStructureForShare(context.getInitialQueryId()));
StoragePtr inner_table = materialized_view->getTargetTable();
auto inner_table_id = inner_table->getStorageID();

View File

@ -358,7 +358,7 @@ void DatabaseMySQL::cleanOutdatedTables()
++iterator;
else
{
const auto table_lock = (*iterator)->lockAlterIntention(RWLockImpl::NO_QUERY);
const auto table_lock = (*iterator)->lockAlterIntention();
(*iterator)->shutdown();
(*iterator)->is_dropped = true;

View File

@ -68,6 +68,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings, const Con
format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter;
format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter;
format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter;
format_settings.regexp.regexp = settings.format_regexp;
format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule;
format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER))
@ -352,10 +355,12 @@ FormatFactory::FormatFactory()
registerOutputFormatProcessorAvro(*this);
registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this);
registerInputFormatProcessorRegexp(*this);
registerFileSegmentationEngineTabSeparated(*this);
registerFileSegmentationEngineCSV(*this);
registerFileSegmentationEngineJSONEachRow(*this);
registerFileSegmentationEngineRegexp(*this);
registerOutputFormatNull(*this);

View File

@ -178,6 +178,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory);
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
void registerFileSegmentationEngineRegexp(FormatFactory & factory);
/// Output only (presentational) formats.
@ -198,5 +199,6 @@ void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory);
/// Input only formats.
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
void registerInputFormatProcessorRegexp(FormatFactory & factory);
}

View File

@ -122,6 +122,15 @@ struct FormatSettings
Avro avro;
struct Regexp
{
std::string regexp;
std::string escaping_rule;
bool skip_unmatched = false;
};
Regexp regexp;
};
}

View File

@ -65,7 +65,7 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName
auto join = storage_join->getJoin();
DataTypes data_types(arguments.size());
auto table_lock = storage_join->lockStructureForShare(false, context.getInitialQueryId());
auto table_lock = storage_join->lockStructureForShare(context.getInitialQueryId());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;

View File

@ -0,0 +1,129 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <common/StringRef.h>
namespace DB
{
/** multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack
* multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found
* multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*/
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template <typename Impl, typename Name>
class FunctionsMultiStringPosition : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringPosition>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
if (src_arr.size() > std::numeric_limits<UInt8>::max())
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
+ ", should be at most 255",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::vector<StringRef> refs;
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size);
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector)
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
size_t refs_size = refs.size();
size_t accum = refs_size;
for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size)
offsets_res[i] = accum;
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
}
};
}

View File

@ -0,0 +1,127 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <common/StringRef.h>
namespace DB
{
/**
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*/
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int FUNCTION_NOT_ALLOWED;
}
/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number.
/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions
template <typename Impl, typename Name, size_t LimitArgs = std::numeric_limits<UInt8>::max()>
class FunctionsMultiStringSearch : public IFunction
{
static_assert(LimitArgs > 0);
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context)
{
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
throw Exception(
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
return std::make_shared<FunctionsMultiStringSearch>();
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return Impl::getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
if (src_arr.size() > LimitArgs)
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
+ ", should be at most " + std::to_string(LimitArgs),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::vector<StringRef> refs;
refs.reserve(src_arr.size());
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
}
};
}

View File

@ -1,5 +1,7 @@
#include "FunctionsStringRegex.h"
#include "FunctionsStringSearch.h"
#include "FunctionsMultiStringSearch.h"
#include "FunctionsStringSearchToString.h"
#include <Columns/ColumnFixedString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <Functions/FunctionFactory.h>

View File

@ -1,707 +0,0 @@
#include "FunctionsStringSearch.h"
#include <algorithm>
#include <memory>
#include <string>
#include <vector>
#include <Columns/ColumnFixedString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/Regexps.h>
#include <IO/WriteHelpers.h>
#include <Poco/UTF8String.h>
#include <Common/Volnitsky.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
*/
struct PositionCaseSensitiveASCII
{
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
using SearcherInBigHaystack = Volnitsky;
/// For search many substrings in one string
using MultiSearcherInBigHaystack = MultiVolnitsky;
/// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization.
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
static size_t countChars(const char * begin, const char * end) { return end - begin; }
/// Convert string to lowercase. Only for case-insensitive search.
/// Implementation is permitted to be inefficient because it is called for single string.
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveASCII
{
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
{
return SearcherInBigHaystack(needle_data, needle_size);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end) { return end - begin; }
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
};
struct PositionCaseSensitiveUTF8
{
using SearcherInBigHaystack = VolnitskyUTF8;
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end)
{
size_t res = 0;
for (auto it = begin; it != end; ++it)
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
++res;
return res;
}
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveUTF8
{
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end)
{
size_t res = 0;
for (auto it = begin; it != end; ++it)
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
++res;
return res;
}
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
};
template <typename Impl>
struct PositionImpl
{
static constexpr bool use_default_implementation_for_constants = false;
using ResultType = UInt64;
/// Find one substring in many strings.
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Current index in the array of strings.
size_t i = 0;
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
/// We will search for the next occurrence in all strings at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = 0;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + needle.size() < begin + offsets[i])
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
else
res[i] = 0;
pos = begin + offsets[i];
++i;
}
if (i < res.size())
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
}
/// Search for substring in string.
static void constantConstant(std::string data, std::string needle, UInt64 & res)
{
Impl::toLowerIfNeed(data);
Impl::toLowerIfNeed(needle);
res = data.find(needle);
if (res == std::string::npos)
res = 0;
else
res = 1 + Impl::countChars(data.data(), data.data() + res);
}
/// Search each time for a different single substring inside each time different string.
static void vectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<UInt64> & res)
{
ColumnString::Offset prev_haystack_offset = 0;
ColumnString::Offset prev_needle_offset = 0;
size_t size = haystack_offsets.size();
for (size_t i = 0; i < size; ++i)
{
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
if (0 == needle_size)
{
/// An empty string is always at the very beginning of `haystack`.
res[i] = 1;
}
else
{
/// It is assumed that the StringSearcher is not very difficult to initialize.
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
/// searcher returns a pointer to the found substring or to the end of `haystack`.
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
- &haystack_data[prev_haystack_offset];
if (pos != haystack_size)
{
res[i] = 1
+ Impl::countChars(
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
}
else
res[i] = 0;
}
prev_haystack_offset = haystack_offsets[i];
prev_needle_offset = needle_offsets[i];
}
}
/// Find many substrings in single string.
static void constantVector(
const String & haystack,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<UInt64> & res)
{
// NOTE You could use haystack indexing. But this is a rare case.
ColumnString::Offset prev_needle_offset = 0;
size_t size = needle_offsets.size();
for (size_t i = 0; i < size; ++i)
{
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
if (0 == needle_size)
{
res[i] = 1;
}
else
{
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
size_t pos = searcher.search(
reinterpret_cast<const UInt8 *>(haystack.data()),
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
- reinterpret_cast<const UInt8 *>(haystack.data());
if (pos != haystack.size())
{
res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos);
}
else
res[i] = 0;
}
prev_needle_offset = needle_offsets[i];
}
}
template <typename... Args>
static void vectorFixedConstant(Args &&...)
{
throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
}
};
template <typename Impl>
struct MultiSearchAllPositionsImpl
{
using ResultType = UInt64;
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
const size_t needles_size = needles.size();
/// Something can be uninitialized after the search itself
std::fill(res.begin(), res.end(), 0);
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback);
prev_offset = haystack_offsets[j];
}
}
}
};
template <typename Impl>
struct MultiSearchImpl
{
using ResultType = UInt8;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt8> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
if (iteration == 0 || !res[j])
res[j] = searcher.searchOne(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
template <typename Impl>
struct MultiSearchFirstPositionImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
if (iteration == 0 || res[j] == 0)
res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
else
{
UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
if (result != 0)
res[j] = std::min(result, res[j]);
}
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
template <typename Impl>
struct MultiSearchFirstIndexImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
/// hasMoreToSearch traverse needles in increasing order
if (iteration == 0 || res[j] == 0)
res[j] = searcher.searchOneFirstIndex(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
*/
template <typename TokenSearcher, bool negate_result = false>
struct HasTokenImpl
{
using ResultType = UInt8;
static constexpr bool use_default_implementation_for_constants = true;
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (offsets.empty())
return;
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// The current index in the array of strings.
size_t i = 0;
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = negate_result;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + pattern.size() < begin + offsets[i])
res[i] = !negate_result;
else
res[i] = negate_result;
pos = begin + offsets[i];
++i;
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0]));
}
template <typename... Args>
static void vectorVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
/// Search different needles in single haystack.
template <typename... Args>
static void constantVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename... Args>
static void vectorFixedConstant(Args &&...)
{
throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
}
};
struct NamePosition
{
static constexpr auto name = "position";
};
struct NamePositionUTF8
{
static constexpr auto name = "positionUTF8";
};
struct NamePositionCaseInsensitive
{
static constexpr auto name = "positionCaseInsensitive";
};
struct NamePositionCaseInsensitiveUTF8
{
static constexpr auto name = "positionCaseInsensitiveUTF8";
};
struct NameMultiSearchAllPositions
{
static constexpr auto name = "multiSearchAllPositions";
};
struct NameMultiSearchAllPositionsUTF8
{
static constexpr auto name = "multiSearchAllPositionsUTF8";
};
struct NameMultiSearchAllPositionsCaseInsensitive
{
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
};
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
};
struct NameMultiSearchAny
{
static constexpr auto name = "multiSearchAny";
};
struct NameMultiSearchAnyUTF8
{
static constexpr auto name = "multiSearchAnyUTF8";
};
struct NameMultiSearchAnyCaseInsensitive
{
static constexpr auto name = "multiSearchAnyCaseInsensitive";
};
struct NameMultiSearchAnyCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
};
struct NameMultiSearchFirstIndex
{
static constexpr auto name = "multiSearchFirstIndex";
};
struct NameMultiSearchFirstIndexUTF8
{
static constexpr auto name = "multiSearchFirstIndexUTF8";
};
struct NameMultiSearchFirstIndexCaseInsensitive
{
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
};
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
};
struct NameMultiSearchFirstPosition
{
static constexpr auto name = "multiSearchFirstPosition";
};
struct NameMultiSearchFirstPositionUTF8
{
static constexpr auto name = "multiSearchFirstPositionUTF8";
};
struct NameMultiSearchFirstPositionCaseInsensitive
{
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
};
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
};
struct NameHasToken
{
static constexpr auto name = "hasToken";
};
struct NameHasTokenCaseInsensitive
{
static constexpr auto name = "hasTokenCaseInsensitive";
};
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
using FunctionPositionCaseInsensitiveUTF8
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
using FunctionMultiSearchAllPositions
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
using FunctionMultiSearchAllPositionsUTF8
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
using FunctionMultiSearchAllPositionsCaseInsensitive
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
using FunctionMultiSearchCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
using FunctionMultiSearchCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstIndex
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
using FunctionMultiSearchFirstIndexUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
using FunctionMultiSearchFirstIndexCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstPosition
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
using FunctionMultiSearchFirstPositionUTF8
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
using FunctionMultiSearchFirstPositionCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
using FunctionHasTokenCaseInsensitive
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
void registerFunctionsStringSearch(FunctionFactory & factory)
{
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionPositionUTF8>();
factory.registerFunction<FunctionPositionCaseInsensitive>();
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
factory.registerFunction<FunctionMultiSearchAllPositions>();
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
factory.registerFunction<FunctionMultiSearch>();
factory.registerFunction<FunctionMultiSearchUTF8>();
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
factory.registerFunction<FunctionMultiSearchFirstIndex>();
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
factory.registerFunction<FunctionMultiSearchFirstPosition>();
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
factory.registerFunction<FunctionHasToken>();
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
}
}

View File

@ -1,6 +1,5 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
@ -10,9 +9,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <common/StringRef.h>
namespace DB
@ -37,40 +34,12 @@ namespace DB
* - the zero subpattern (the match part, otherwise);
* - if not match - an empty string.
* extract(haystack, pattern)
*
* 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) - replaces the pattern with the specified regexp, only the first occurrence.
* replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences.
*
* multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack
* multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found
* multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*/
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int FUNCTION_NOT_ALLOWED;
}
template <typename Impl, typename Name>
@ -163,224 +132,4 @@ public:
}
};
template <typename Impl, typename Name>
class FunctionsStringSearchToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isString(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
if (!col_needle)
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnString::create();
ColumnString::Chars & vec_res = col_res->getChars();
ColumnString::Offsets & offsets_res = col_res->getOffsets();
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
block.getByPosition(result).column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
template <typename Impl, typename Name>
class FunctionsMultiStringPosition : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringPosition>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
if (src_arr.size() > std::numeric_limits<UInt8>::max())
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
+ ", should be at most 255",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::vector<StringRef> refs;
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size);
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector)
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
size_t refs_size = refs.size();
size_t accum = refs_size;
for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size)
offsets_res[i] = accum;
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
}
};
/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number.
/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions
template <typename Impl, typename Name, size_t LimitArgs = std::numeric_limits<UInt8>::max()>
class FunctionsMultiStringSearch : public IFunction
{
static_assert(LimitArgs > 0);
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context)
{
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
throw Exception(
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
return std::make_shared<FunctionsMultiStringSearch>();
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return Impl::getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
if (src_arr.size() > LimitArgs)
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
+ ", should be at most " + std::to_string(LimitArgs),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::vector<StringRef> refs;
refs.reserve(src_arr.size());
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
}
};
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <common/StringRef.h>
namespace DB
{
/** Applies regexp re2 and extracts:
* - 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)
*/
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
template <typename Impl, typename Name>
class FunctionsStringSearchToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isString(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
if (!col_needle)
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnString::create();
ColumnString::Chars & vec_res = col_res->getChars();
ColumnString::Offsets & offsets_res = col_res->getOffsets();
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
block.getByPosition(result).column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
}

View File

@ -0,0 +1,83 @@
#pragma once
#include <Columns/ColumnString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
*/
template <typename TokenSearcher, bool negate_result = false>
struct HasTokenImpl
{
using ResultType = UInt8;
static constexpr bool use_default_implementation_for_constants = true;
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (offsets.empty())
return;
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// The current index in the array of strings.
size_t i = 0;
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = negate_result;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + pattern.size() < begin + offsets[i])
res[i] = !negate_result;
else
res[i] = negate_result;
pos = begin + offsets[i];
++i;
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0]));
}
template <typename... Args>
static void vectorVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
/// Search different needles in single haystack.
template <typename... Args>
static void constantVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename... Args>
static void vectorFixedConstant(Args &&...)
{
throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
}
};
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <vector>
#include <Columns/ColumnString.h>
namespace DB
{
template <typename Impl>
struct MultiSearchAllPositionsImpl
{
using ResultType = UInt64;
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
const size_t needles_size = needles.size();
/// Something can be uninitialized after the search itself
std::fill(res.begin(), res.end(), 0);
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback);
prev_offset = haystack_offsets[j];
}
}
}
};
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <vector>
#include <Columns/ColumnString.h>
namespace DB
{
template <typename Impl>
struct MultiSearchFirstIndexImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
/// hasMoreToSearch traverse needles in increasing order
if (iteration == 0 || res[j] == 0)
res[j] = searcher.searchOneFirstIndex(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
}

View File

@ -0,0 +1,57 @@
#pragma once
#include <vector>
#include <Columns/ColumnString.h>
namespace DB
{
template <typename Impl>
struct MultiSearchFirstPositionImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
if (iteration == 0 || res[j] == 0)
res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
else
{
UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
if (result != 0)
res[j] = std::min(result, res[j]);
}
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <vector>
#include <Columns/ColumnString.h>
namespace DB
{
template <typename Impl>
struct MultiSearchImpl
{
using ResultType = UInt8;
static constexpr bool is_using_hyperscan = false;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt8> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
size_t prev_offset = 0;
for (size_t j = 0; j < haystack_string_size; ++j)
{
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
if (iteration == 0 || !res[j])
res[j] = searcher.searchOne(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
++iteration;
}
}
};
}

View File

@ -0,0 +1,307 @@
#include "FunctionsStringSearch.h"
#include <string>
#include <vector>
#include <Poco/UTF8String.h>
#include <Common/Volnitsky.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
*/
struct PositionCaseSensitiveASCII
{
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
using SearcherInBigHaystack = Volnitsky;
/// For search many substrings in one string
using MultiSearcherInBigHaystack = MultiVolnitsky;
/// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization.
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
static size_t countChars(const char * begin, const char * end) { return end - begin; }
/// Convert string to lowercase. Only for case-insensitive search.
/// Implementation is permitted to be inefficient because it is called for single string.
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveASCII
{
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
{
return SearcherInBigHaystack(needle_data, needle_size);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end) { return end - begin; }
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
};
struct PositionCaseSensitiveUTF8
{
using SearcherInBigHaystack = VolnitskyUTF8;
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end)
{
size_t res = 0;
for (auto it = begin; it != end; ++it)
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
++res;
return res;
}
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveUTF8
{
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
}
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
{
return SearcherInSmallHaystack(needle_data, needle_size);
}
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{
return MultiSearcherInBigHaystack(needles);
}
static size_t countChars(const char * begin, const char * end)
{
size_t res = 0;
for (auto it = begin; it != end; ++it)
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
++res;
return res;
}
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
};
template <typename Impl>
struct PositionImpl
{
static constexpr bool use_default_implementation_for_constants = false;
using ResultType = UInt64;
/// Find one substring in many strings.
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Current index in the array of strings.
size_t i = 0;
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
/// We will search for the next occurrence in all strings at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = 0;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + needle.size() < begin + offsets[i])
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
else
res[i] = 0;
pos = begin + offsets[i];
++i;
}
if (i < res.size())
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
}
/// Search for substring in string.
static void constantConstant(std::string data, std::string needle, UInt64 & res)
{
Impl::toLowerIfNeed(data);
Impl::toLowerIfNeed(needle);
res = data.find(needle);
if (res == std::string::npos)
res = 0;
else
res = 1 + Impl::countChars(data.data(), data.data() + res);
}
/// Search each time for a different single substring inside each time different string.
static void vectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<UInt64> & res)
{
ColumnString::Offset prev_haystack_offset = 0;
ColumnString::Offset prev_needle_offset = 0;
size_t size = haystack_offsets.size();
for (size_t i = 0; i < size; ++i)
{
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
if (0 == needle_size)
{
/// An empty string is always at the very beginning of `haystack`.
res[i] = 1;
}
else
{
/// It is assumed that the StringSearcher is not very difficult to initialize.
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
/// searcher returns a pointer to the found substring or to the end of `haystack`.
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
- &haystack_data[prev_haystack_offset];
if (pos != haystack_size)
{
res[i] = 1
+ Impl::countChars(
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
}
else
res[i] = 0;
}
prev_haystack_offset = haystack_offsets[i];
prev_needle_offset = needle_offsets[i];
}
}
/// Find many substrings in single string.
static void constantVector(
const String & haystack,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<UInt64> & res)
{
// NOTE You could use haystack indexing. But this is a rare case.
ColumnString::Offset prev_needle_offset = 0;
size_t size = needle_offsets.size();
for (size_t i = 0; i < size; ++i)
{
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
if (0 == needle_size)
{
res[i] = 1;
}
else
{
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
size_t pos = searcher.search(
reinterpret_cast<const UInt8 *>(haystack.data()),
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
- reinterpret_cast<const UInt8 *>(haystack.data());
if (pos != haystack.size())
{
res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos);
}
else
res[i] = 0;
}
prev_needle_offset = needle_offsets[i];
}
}
template <typename... Args>
static void vectorFixedConstant(Args &&...)
{
throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
}
};
}

View File

@ -1,5 +1,5 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringSearch.h>
#include <Functions/FunctionsStringSearchToString.h>
#include <common/find_symbols.h>
namespace DB

View File

@ -1,5 +1,5 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringSearch.h>
#include <Functions/FunctionsStringSearchToString.h>
#include <common/find_symbols.h>
namespace DB

View File

@ -5,12 +5,10 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnAggregateFunction.h>
#include <IO/WriteHelpers.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/AlignedBuffer.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
@ -108,7 +106,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
IAggregateFunction & agg_func = *aggregate_function.get();
IAggregateFunction & agg_func = *aggregate_function;
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
@ -132,7 +130,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
else if (const ColumnConst * const_arr = checkAndGetColumnConst<ColumnArray>(col))
{
materialized_columns.emplace_back(const_arr->convertToFullColumn());
const auto & materialized_arr = typeid_cast<const ColumnArray &>(*materialized_columns.back().get());
const auto & materialized_arr = typeid_cast<const ColumnArray &>(*materialized_columns.back());
aggregate_arguments_vec[i] = &materialized_arr.getData();
offsets_i = &materialized_arr.getOffsets();
}

View File

@ -0,0 +1,394 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
/** Applies an aggregate function to value ranges in the array.
* The function does what arrayReduce do on a structure similar to segment tree.
* Space complexity: n * log(n)
*
* arrayReduceInRanges('agg', indices, lengths, arr1, ...)
*/
class FunctionArrayReduceInRanges : public IFunction
{
public:
static const size_t minimum_step = 64;
static constexpr auto name = "arrayReduceInRanges";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayReduceInRanges>(); }
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
/// lazy initialization in getReturnTypeImpl
/// TODO: init in OverloadResolver
mutable AggregateFunctionPtr aggregate_function;
};
DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
/// The first argument is a constant string with the name of the aggregate function
/// (possibly with parameters in parentheses, for example: "quantile(0.99)").
if (arguments.size() < 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be at least 3.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
if (!aggregate_function_name_column)
throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * ranges_type_array = checkAndGetDataType<DataTypeArray>(arguments[1].type.get());
if (!ranges_type_array)
throw Exception("Second argument for function " + getName() + " must be an array of ranges.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeTuple * ranges_type_tuple = checkAndGetDataType<DataTypeTuple>(ranges_type_array->getNestedType().get());
if (!ranges_type_tuple || ranges_type_tuple->getElements().size() != 2)
throw Exception("Each array element in the second argument for function " + getName() + " must be a tuple (index, length).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isNativeInteger(ranges_type_tuple->getElements()[0]))
throw Exception("First tuple member in the second argument for function " + getName() + " must be ints or uints.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!WhichDataType(ranges_type_tuple->getElements()[1]).isNativeUInt())
throw Exception("Second tuple member in the second argument for function " + getName() + " must be uints.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
DataTypes argument_types(arguments.size() - 2);
for (size_t i = 2, size = arguments.size(); i < size; ++i)
{
const DataTypeArray * arg = checkAndGetDataType<DataTypeArray>(arguments[i].type.get());
if (!arg)
throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type "
+ arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
argument_types[i - 2] = arg->getNestedType();
}
if (!aggregate_function)
{
String aggregate_function_name_with_params = aggregate_function_name_column->getValue<String>();
if (aggregate_function_name_with_params.empty())
throw Exception("First argument for function " + getName() + " (name of aggregate function) cannot be empty.",
ErrorCodes::BAD_ARGUMENTS);
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params,
aggregate_function_name, params_row, "function " + getName());
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row);
}
return std::make_shared<DataTypeArray>(aggregate_function->getReturnType());
}
void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
IAggregateFunction & agg_func = *aggregate_function;
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
std::vector<ColumnPtr> materialized_columns;
/// Handling ranges
const IColumn * ranges_col_array = block.getByPosition(arguments[1]).column.get();
const IColumn * ranges_col_tuple = nullptr;
const ColumnArray::Offsets * ranges_offsets = nullptr;
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(ranges_col_array))
{
ranges_col_tuple = &arr->getData();
ranges_offsets = &arr->getOffsets();
}
else if (const ColumnConst * const_arr = checkAndGetColumnConst<ColumnArray>(ranges_col_array))
{
materialized_columns.emplace_back(const_arr->convertToFullColumn());
const auto & materialized_arr = typeid_cast<const ColumnArray &>(*materialized_columns.back());
ranges_col_tuple = &materialized_arr.getData();
ranges_offsets = &materialized_arr.getOffsets();
}
else
throw Exception("Illegal column " + ranges_col_array->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
const IColumn & indices_col = static_cast<const ColumnTuple *>(ranges_col_tuple)->getColumn(0);
const IColumn & lengths_col = static_cast<const ColumnTuple *>(ranges_col_tuple)->getColumn(1);
/// Handling arguments
/// The code is mostly copied from `arrayReduce`. Maybe create a utility header?
const size_t num_arguments_columns = arguments.size() - 2;
std::vector<const IColumn *> aggregate_arguments_vec(num_arguments_columns);
const ColumnArray::Offsets * offsets = nullptr;
for (size_t i = 0; i < num_arguments_columns; ++i)
{
const IColumn * col = block.getByPosition(arguments[i + 2]).column.get();
const ColumnArray::Offsets * offsets_i = nullptr;
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
{
aggregate_arguments_vec[i] = &arr->getData();
offsets_i = &arr->getOffsets();
}
else if (const ColumnConst * const_arr = checkAndGetColumnConst<ColumnArray>(col))
{
materialized_columns.emplace_back(const_arr->convertToFullColumn());
const auto & materialized_arr = typeid_cast<const ColumnArray &>(*materialized_columns.back());
aggregate_arguments_vec[i] = &materialized_arr.getData();
offsets_i = &materialized_arr.getOffsets();
}
else
throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
if (i == 0)
offsets = offsets_i;
else if (*offsets_i != *offsets)
throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.",
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
}
const IColumn ** aggregate_arguments = aggregate_arguments_vec.data();
/// Handling results
MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn();
ColumnArray * result_arr = static_cast<ColumnArray *>(result_holder.get());
IColumn & result_data = result_arr->getData();
result_arr->getOffsets().insert(ranges_offsets->begin(), ranges_offsets->end());
/// AggregateFunction's states should be inserted into column using specific way
auto res_col_aggregate_function = typeid_cast<ColumnAggregateFunction *>(&result_data);
if (!res_col_aggregate_function && agg_func.isState())
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
+ block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN);
/// Perform the aggregation
size_t begin = 0;
size_t end = 0;
size_t ranges_begin = 0;
size_t ranges_end = 0;
for (size_t i = 0; i < input_rows_count; ++i)
{
begin = end;
end = (*offsets)[i];
ranges_begin = ranges_end;
ranges_end = (*ranges_offsets)[i];
/// We will allocate pre-aggregation places for each `minimum_place << level` rows.
/// The value of `level` starts from 0, and it will never exceed the number of bits in a `size_t`.
/// We calculate the offset (and thus size) of those places in each level.
size_t place_offsets[sizeof(size_t) * 8];
size_t place_total = 0;
{
size_t place_in_level = (end - begin) / minimum_step;
place_offsets[0] = place_in_level;
for (size_t level = 0; place_in_level; ++level)
{
place_in_level >>= 1;
place_total = place_offsets[level] + place_in_level;
place_offsets[level + 1] = place_total;
}
}
PODArray<AggregateDataPtr> places(place_total);
for (size_t j = 0; j < place_total; ++j)
{
places[j] = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData());
try
{
agg_func.create(places[j]);
}
catch (...)
{
for (size_t k = 0; k < j; ++k)
agg_func.destroy(places[k]);
throw;
}
}
SCOPE_EXIT({
for (size_t j = 0; j < place_total; ++j)
agg_func.destroy(places[j]);
});
auto true_func = &agg_func;
/// Unnest consecutive trailing -State combinators
while (auto func = typeid_cast<AggregateFunctionState *>(true_func))
true_func = func->getNestedFunction().get();
/// Pre-aggregate to the initial level
for (size_t j = 0; j < place_offsets[0]; ++j)
{
size_t local_begin = j * minimum_step;
size_t local_end = (j + 1) * minimum_step;
for (size_t k = local_begin; k < local_end; ++k)
true_func->add(places[j], aggregate_arguments, begin + k, arena.get());
}
/// Pre-aggregate to the higher levels by merging
{
size_t place_in_level = place_offsets[0] >> 1;
size_t place_begin = 0;
for (size_t level = 0; place_in_level; ++level)
{
size_t next_place_begin = place_offsets[level];
for (size_t j = 0; j < place_in_level; ++j)
{
true_func->merge(places[next_place_begin + j], places[place_begin + (j << 1)], arena.get());
true_func->merge(places[next_place_begin + j], places[place_begin + (j << 1) + 1], arena.get());
}
place_in_level >>= 1;
place_begin = next_place_begin;
}
}
for (size_t j = ranges_begin; j < ranges_end; ++j)
{
size_t local_begin = 0;
size_t local_end = 0;
{
Int64 index = indices_col.getInt(j);
UInt64 length = lengths_col.getUInt(j);
/// Keep the same as in arraySlice
if (index > 0)
{
local_begin = index - 1;
if (local_begin + length < end - begin)
local_end = local_begin + length;
else
local_end = end - begin;
}
else if (index < 0)
{
if (end - begin + index > 0)
local_begin = end - begin + index;
else
local_begin = 0;
if (local_begin + length < end - begin)
local_end = local_begin + length;
else
local_end = end - begin;
}
}
size_t place_begin = (local_begin + minimum_step - 1) / minimum_step;
size_t place_end = local_end / minimum_step;
AggregateDataPtr place = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData());
agg_func.create(place);
SCOPE_EXIT({
agg_func.destroy(place);
});
if (place_begin < place_end)
{
/// In this case, we can use pre-aggregated data.
/// Aggregate rows before
for (size_t k = local_begin; k < place_begin * minimum_step; ++k)
true_func->add(place, aggregate_arguments, begin + k, arena.get());
/// Aggregate using pre-aggretated data
{
size_t level = 0;
size_t place_curr = place_begin;
while (place_curr < place_end)
{
while (((place_curr >> level) & 1) == 0 && place_curr + (2 << level) <= place_end)
level += 1;
while (place_curr + (1 << level) > place_end)
level -= 1;
size_t place_offset = 0;
if (level)
place_offset = place_offsets[level - 1];
true_func->merge(place, places[place_offset + (place_curr >> level)], arena.get());
place_curr += 1 << level;
}
}
/// Aggregate rows after
for (size_t k = place_end * minimum_step; k < local_end; ++k)
true_func->add(place, aggregate_arguments, begin + k, arena.get());
}
else
{
/// In this case, we can not use pre-aggregated data.
for (size_t k = local_begin; k < local_end; ++k)
true_func->add(place, aggregate_arguments, begin + k, arena.get());
}
if (!res_col_aggregate_function)
agg_func.insertResultInto(place, result_data);
else
res_col_aggregate_function->insertFrom(place);
}
}
block.getByPosition(result).column = std::move(result_holder);
}
void registerFunctionArrayReduceInRanges(FunctionFactory & factory)
{
factory.registerFunction<FunctionArrayReduceInRanges>();
}
}

View File

@ -33,6 +33,7 @@ void registerFunctionArrayFlatten(FunctionFactory &);
void registerFunctionArrayWithConstant(FunctionFactory &);
void registerFunctionArrayZip(FunctionFactory &);
void registerFunctionArrayAUC(FunctionFactory &);
void registerFunctionArrayReduceInRanges(FunctionFactory &);
void registerFunctionsArray(FunctionFactory & factory)
{
@ -53,6 +54,7 @@ void registerFunctionsArray(FunctionFactory & factory)
registerFunctionArraySlice(factory);
registerFunctionArrayReverse(factory);
registerFunctionArrayReduce(factory);
registerFunctionArrayReduceInRanges(factory);
registerFunctionRange(factory);
registerFunctionsEmptyArray(factory);
registerFunctionEmptyArrayToSingle(factory);

View File

@ -0,0 +1,22 @@
#include "FunctionsStringSearch.h"
#include <Functions/FunctionFactory.h>
#include "HasTokenImpl.h"
#include <Common/Volnitsky.h>
namespace DB
{
struct NameHasToken
{
static constexpr auto name = "hasToken";
};
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
void registerFunctionHasToken(FunctionFactory & factory)
{
factory.registerFunction<FunctionHasToken>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsStringSearch.h"
#include <Functions/FunctionFactory.h>
#include "HasTokenImpl.h"
#include <Common/Volnitsky.h>
namespace DB
{
struct NameHasTokenCaseInsensitive
{
static constexpr auto name = "hasTokenCaseInsensitive";
};
using FunctionHasTokenCaseInsensitive
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
void registerFunctionHasTokenCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringPosition.h"
#include "FunctionFactory.h"
#include "MultiSearchAllPositionsImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAllPositions
{
static constexpr auto name = "multiSearchAllPositions";
};
using FunctionMultiSearchAllPositions
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
void registerFunctionMultiSearchAllPositions(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchAllPositions>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringPosition.h"
#include "FunctionFactory.h"
#include "MultiSearchAllPositionsImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAllPositionsCaseInsensitive
{
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
};
using FunctionMultiSearchAllPositionsCaseInsensitive
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
}
}

View File

@ -0,0 +1,24 @@
#include "FunctionsMultiStringPosition.h"
#include "FunctionFactory.h"
#include "MultiSearchAllPositionsImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
};
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringPosition.h"
#include "FunctionFactory.h"
#include "MultiSearchAllPositionsImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAllPositionsUTF8
{
static constexpr auto name = "multiSearchAllPositionsUTF8";
};
using FunctionMultiSearchAllPositionsUTF8
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
}
}

View File

@ -0,0 +1,22 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAny
{
static constexpr auto name = "multiSearchAny";
};
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
void registerFunctionMultiSearchAny(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearch>();
}
}

View File

@ -0,0 +1,22 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAnyCaseInsensitive
{
static constexpr auto name = "multiSearchAnyCaseInsensitive";
};
using FunctionMultiSearchCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAnyCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
};
using FunctionMultiSearchCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
}
}

View File

@ -0,0 +1,21 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchAnyUTF8
{
static constexpr auto name = "multiSearchAnyUTF8";
};
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
void registerFunctionMultiSearchAnyUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchUTF8>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstIndexImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstIndex
{
static constexpr auto name = "multiSearchFirstIndex";
};
using FunctionMultiSearchFirstIndex
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
void registerFunctionMultiSearchFirstIndex(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstIndex>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstIndexImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstIndexCaseInsensitive
{
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
};
using FunctionMultiSearchFirstIndexCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstIndexImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
};
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstIndexImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstIndexUTF8
{
static constexpr auto name = "multiSearchFirstIndexUTF8";
};
using FunctionMultiSearchFirstIndexUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstPositionImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstPosition
{
static constexpr auto name = "multiSearchFirstPosition";
};
using FunctionMultiSearchFirstPosition
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
void registerFunctionMultiSearchFirstPosition(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstPosition>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstPositionImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstPositionCaseInsensitive
{
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
};
using FunctionMultiSearchFirstPositionCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
}
}

View File

@ -0,0 +1,24 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstPositionImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
{
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
};
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
}
}

View File

@ -0,0 +1,23 @@
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiSearchFirstPositionImpl.h"
#include "PositionImpl.h"
namespace DB
{
struct NameMultiSearchFirstPositionUTF8
{
static constexpr auto name = "multiSearchFirstPositionUTF8";
};
using FunctionMultiSearchFirstPositionUTF8
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
}
}

View File

@ -0,0 +1,21 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "PositionImpl.h"
namespace DB
{
struct NamePosition
{
static constexpr auto name = "position";
};
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
void registerFunctionPosition(FunctionFactory & factory)
{
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,20 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "PositionImpl.h"
namespace DB
{
struct NamePositionCaseInsensitive
{
static constexpr auto name = "positionCaseInsensitive";
};
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
void registerFunctionPositionCaseInsensitive(FunctionFactory & factory)
{
factory.registerFunction<FunctionPositionCaseInsensitive>();
}
}

View File

@ -0,0 +1,22 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "PositionImpl.h"
namespace DB
{
struct NamePositionCaseInsensitiveUTF8
{
static constexpr auto name = "positionCaseInsensitiveUTF8";
};
using FunctionPositionCaseInsensitiveUTF8
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
}
}

View File

@ -0,0 +1,21 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "PositionImpl.h"
namespace DB
{
struct NamePositionUTF8
{
static constexpr auto name = "positionUTF8";
};
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
void registerFunctionPositionUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionPositionUTF8>();
}
}

View File

@ -0,0 +1,66 @@
namespace DB
{
class FunctionFactory;
void registerFunctionPosition(FunctionFactory &);
void registerFunctionPositionUTF8(FunctionFactory &);
void registerFunctionPositionCaseInsensitive(FunctionFactory &);
void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory &);
void registerFunctionMultiSearchAny(FunctionFactory &);
void registerFunctionMultiSearchAnyUTF8(FunctionFactory &);
void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory &);
void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory &);
void registerFunctionMultiSearchFirstIndex(FunctionFactory &);
void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory &);
void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory &);
void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory &);
void registerFunctionMultiSearchFirstPosition(FunctionFactory &);
void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory &);
void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory &);
void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory &);
void registerFunctionMultiSearchAllPositions(FunctionFactory &);
void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory &);
void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory &);
void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory &);
void registerFunctionHasToken(FunctionFactory &);
void registerFunctionHasTokenCaseInsensitive(FunctionFactory &);
void registerFunctionsStringSearch(FunctionFactory & factory)
{
registerFunctionPosition(factory);
registerFunctionPositionUTF8(factory);
registerFunctionPositionCaseInsensitive(factory);
registerFunctionPositionCaseInsensitiveUTF8(factory);
registerFunctionMultiSearchAny(factory);
registerFunctionMultiSearchAnyUTF8(factory);
registerFunctionMultiSearchAnyCaseInsensitive(factory);
registerFunctionMultiSearchAnyCaseInsensitiveUTF8(factory);
registerFunctionMultiSearchFirstIndex(factory);
registerFunctionMultiSearchFirstIndexUTF8(factory);
registerFunctionMultiSearchFirstIndexCaseInsensitive(factory);
registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(factory);
registerFunctionMultiSearchFirstPosition(factory);
registerFunctionMultiSearchFirstPositionUTF8(factory);
registerFunctionMultiSearchFirstPositionCaseInsensitive(factory);
registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(factory);
registerFunctionMultiSearchAllPositions(factory);
registerFunctionMultiSearchAllPositionsUTF8(factory);
registerFunctionMultiSearchAllPositionsCaseInsensitive(factory);
registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(factory);
registerFunctionHasToken(factory);
registerFunctionHasTokenCaseInsensitive(factory);
}
}

View File

@ -1,6 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
#include <Functions/FunctionsStringSearchToString.h>
namespace DB

View File

@ -1,6 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
#include <Functions/FunctionsStringSearchToString.h>
namespace DB

View File

@ -24,7 +24,6 @@
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/Set.h>

View File

@ -38,6 +38,7 @@ struct NonGlobalTableData
const CheckShardsAndTables & checker;
const Context & context;
std::vector<ASTPtr> & renamed_tables;
ASTFunction * function = nullptr;
ASTTableJoin * table_join = nullptr;
@ -95,10 +96,11 @@ private:
String alias = database_and_table->tryGetAlias();
if (alias.empty())
throw Exception("Distributed table should have an alias when distributed_product_mode set to local.",
throw Exception("Distributed table should have an alias when distributed_product_mode set to local",
ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED);
auto & identifier = database_and_table->as<ASTIdentifier &>();
renamed_tables.emplace_back(identifier.clone());
identifier.resetTable(database, table);
}
else
@ -118,6 +120,7 @@ public:
{
const CheckShardsAndTables & checker;
const Context & context;
std::vector<std::pair<ASTPtr, std::vector<ASTPtr>>> & renamed_tables;
};
static void visit(ASTPtr & node, Data & data)
@ -148,8 +151,11 @@ private:
if (node.name == "in" || node.name == "notIn")
{
auto & subquery = node.arguments->children.at(1);
NonGlobalTableVisitor::Data table_data{data.checker, data.context, &node, nullptr};
std::vector<ASTPtr> renamed;
NonGlobalTableVisitor::Data table_data{data.checker, data.context, renamed, &node, nullptr};
NonGlobalTableVisitor(table_data).visit(subquery);
if (!renamed.empty())
data.renamed_tables.emplace_back(subquery, std::move(renamed));
}
}
@ -163,8 +169,11 @@ private:
{
if (auto & subquery = node.table_expression->as<ASTTableExpression>()->subquery)
{
NonGlobalTableVisitor::Data table_data{data.checker, data.context, nullptr, table_join};
std::vector<ASTPtr> renamed;
NonGlobalTableVisitor::Data table_data{data.checker, data.context, renamed, nullptr, table_join};
NonGlobalTableVisitor(table_data).visit(subquery);
if (!renamed.empty())
data.renamed_tables.emplace_back(subquery, std::move(renamed));
}
}
}
@ -208,7 +217,7 @@ void InJoinSubqueriesPreprocessor::visit(ASTPtr & ast) const
return;
}
NonGlobalSubqueryVisitor::Data visitor_data{*checker, context};
NonGlobalSubqueryVisitor::Data visitor_data{*checker, context, renamed_tables};
NonGlobalSubqueryVisitor(visitor_data).visit(ast);
}

View File

@ -35,6 +35,8 @@ class Context;
class InJoinSubqueriesPreprocessor
{
public:
using SubqueryTables = std::vector<std::pair<ASTPtr, std::vector<ASTPtr>>>; /// {subquery, renamed_tables}
struct CheckShardsAndTables
{
using Ptr = std::unique_ptr<CheckShardsAndTables>;
@ -45,8 +47,10 @@ public:
virtual ~CheckShardsAndTables() {}
};
InJoinSubqueriesPreprocessor(const Context & context_, CheckShardsAndTables::Ptr _checker = std::make_unique<CheckShardsAndTables>())
InJoinSubqueriesPreprocessor(const Context & context_, SubqueryTables & renamed_tables_,
CheckShardsAndTables::Ptr _checker = std::make_unique<CheckShardsAndTables>())
: context(context_)
, renamed_tables(renamed_tables_)
, checker(std::move(_checker))
{}
@ -54,6 +58,7 @@ public:
private:
const Context & context;
SubqueryTables & renamed_tables;
CheckShardsAndTables::Ptr checker;
};

View File

@ -82,7 +82,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!mutation_commands.empty())
{
auto table_lock_holder = table->lockStructureForShare(false /* because mutation is executed asyncronously */, context.getCurrentQueryId());
auto table_lock_holder = table->lockStructureForShare(context.getCurrentQueryId());
MutationsInterpreter(table, mutation_commands, context, false).validate(table_lock_holder);
table->mutate(mutation_commands, context);
}
@ -101,7 +101,7 @@ BlockIO InterpreterAlterQuery::execute()
switch (command.type)
{
case LiveViewCommand::REFRESH:
live_view->refresh(context);
live_view->refresh();
break;
}
}
@ -109,7 +109,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!alter_commands.empty())
{
auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId());
auto table_lock_holder = table->lockAlterIntention();
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
alter_commands.validate(metadata, context);
alter_commands.prepare(metadata);

View File

@ -411,7 +411,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table});
/// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table.
as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId());
as_storage_lock = as_storage->lockStructureForShare(context.getCurrentQueryId());
properties.columns = as_storage->getColumns();
/// Secondary indices make sense only for MergeTree family of storage engines.

View File

@ -89,7 +89,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
table = DatabaseCatalog::instance().getTable(table_id);
}
auto table_lock = table->lockStructureForShare(false, context.getInitialQueryId());
auto table_lock = table->lockStructureForShare(context.getInitialQueryId());
columns = table->getColumns();
}

View File

@ -109,7 +109,7 @@ BlockIO InterpreterInsertQuery::execute()
BlockIO res;
StoragePtr table = getTable(query);
auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId());
auto table_lock = table->lockStructureForShare(context.getInitialQueryId());
auto query_sample_block = getSampleBlock(query, table);
if (!query.table_function)

View File

@ -255,7 +255,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (storage)
{
table_lock = storage->lockStructureForShare(false, context->getInitialQueryId());
table_lock = storage->lockStructureForShare(context->getInitialQueryId());
table_id = storage->getStorageID();
}
@ -301,6 +301,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = interpreter_subquery->getSampleBlock();
}
joined_tables.rewriteDistributedInAndJoins(query_ptr);
max_streams = settings.max_threads;
ASTSelectQuery & query = getSelectQuery();
@ -508,7 +510,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl(bool try_move_to_prewhere)
}
if (storage && !options.only_analyze)
from_stage = storage->getQueryProcessingStage(*context);
from_stage = storage->getQueryProcessingStage(*context, query_ptr);
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
bool first_stage = from_stage < QueryProcessingStage::WithMergeableState
@ -1062,7 +1064,7 @@ void InterpreterSelectQuery::executeFetchColumns(
auto check_trivial_count_query = [&]() -> std::optional<AggregateDescription>
{
if (!settings.optimize_trivial_count_query || !syntax_analyzer_result->maybe_optimize_trivial_count || !storage
|| query.sampleSize() || query.sampleOffset() || query.final() || query.prewhere() || query.where()
|| query.sampleSize() || query.sampleOffset() || query.final() || query.prewhere() || query.where() || query.groupBy()
|| !query_analyzer->hasAggregation() || processing_stage != QueryProcessingStage::FetchColumns)
return {};

View File

@ -1,12 +1,18 @@
#include <Interpreters/JoinedTables.h>
#include <Interpreters/Context.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Storages/IStorage.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageValues.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTQualifiedAsterisk.h>
namespace DB
{
@ -14,6 +20,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ALIAS_REQUIRED;
extern const int AMBIGUOUS_COLUMN_NAME;
}
namespace
@ -32,6 +39,71 @@ void checkTablesWithColumns(const std::vector<T> & tables_with_columns, const Co
}
}
class RenameQualifiedIdentifiersMatcher
{
public:
using Data = const std::vector<DatabaseAndTableWithAlias>;
static void visit(ASTPtr & ast, Data & data)
{
if (auto * t = ast->as<ASTIdentifier>())
visit(*t, ast, data);
if (auto * node = ast->as<ASTQualifiedAsterisk>())
visit(*node, ast, data);
}
static bool needChildVisit(ASTPtr & node, const ASTPtr & child)
{
if (node->as<ASTTableExpression>() ||
node->as<ASTQualifiedAsterisk>() ||
child->as<ASTSubquery>())
return false; // NOLINT
return true;
}
private:
static void visit(ASTIdentifier & identifier, ASTPtr &, Data & data)
{
if (identifier.isShort())
return;
bool rewritten = false;
for (auto & table : data)
{
/// Table has an alias. We do not need to rewrite qualified names with table alias (match == ColumnMatch::TableName).
auto match = IdentifierSemantic::canReferColumnToTable(identifier, table);
if (match == IdentifierSemantic::ColumnMatch::AliasedTableName ||
match == IdentifierSemantic::ColumnMatch::DbAndTable)
{
if (rewritten)
throw Exception("Failed to rewrite distributed table names. Ambiguous column '" + identifier.name + "'",
ErrorCodes::AMBIGUOUS_COLUMN_NAME);
/// Table has an alias. So we set a new name qualified by table alias.
IdentifierSemantic::setColumnLongName(identifier, table);
rewritten = true;
}
}
}
static void visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
{
ASTIdentifier & identifier = *node.children[0]->as<ASTIdentifier>();
bool rewritten = false;
for (auto & table : data)
{
if (identifier.name == table.table)
{
if (rewritten)
throw Exception("Failed to rewrite distributed table. Ambiguous column '" + identifier.name + "'",
ErrorCodes::AMBIGUOUS_COLUMN_NAME);
identifier.setShortName(table.alias);
rewritten = true;
}
}
}
};
using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitor<RenameQualifiedIdentifiersMatcher, true>;
}
JoinedTables::JoinedTables(Context && context_, const ASTSelectQuery & select_query)
@ -114,4 +186,27 @@ void JoinedTables::makeFakeTable(StoragePtr storage, const Block & source_header
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, source_header.getNamesAndTypesList());
}
void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query)
{
/// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting.
InJoinSubqueriesPreprocessor::SubqueryTables renamed_tables;
InJoinSubqueriesPreprocessor(context, renamed_tables).visit(query);
String database;
if (!renamed_tables.empty())
database = context.getCurrentDatabase();
for (auto & [subquery, ast_tables] : renamed_tables)
{
std::vector<DatabaseAndTableWithAlias> renamed;
renamed.reserve(ast_tables.size());
for (auto & ast : ast_tables)
renamed.emplace_back(DatabaseAndTableWithAlias(*ast->as<ASTIdentifier>(), database));
/// Change qualified column names in distributed subqueries using table aliases.
RenameQualifiedIdentifiersVisitor::Data data(renamed);
RenameQualifiedIdentifiersVisitor(data).visit(subquery);
}
}
}

View File

@ -37,6 +37,8 @@ public:
const StorageID & leftTableID() const { return table_id; }
void rewriteDistributedInAndJoins(ASTPtr & query);
std::unique_ptr<InterpreterSelectWithUnionQuery> makeLeftTableSubquery(const SelectQueryOptions & select_options);
private:

View File

@ -558,6 +558,10 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &
all_asts->children.push_back(std::make_shared<ASTIdentifier>(column));
auto syntax_result = SyntaxAnalyzer(context).analyze(all_asts, all_columns);
if (context.hasQueryContext())
for (const auto & it : syntax_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second);
stage.analyzer = std::make_unique<ExpressionAnalyzer>(all_asts, syntax_result, context);
ExpressionActionsChain & actions_chain = stage.expressions_chain;

View File

@ -47,7 +47,7 @@ private:
StoragePtr storage;
MutationCommands commands;
const Context & context;
Context context;
bool can_execute;
ASTPtr mutation_ast;

View File

@ -2,7 +2,6 @@
#include <Core/NamesAndTypes.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
#include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
@ -819,9 +818,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_column_names);
/// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting.
InJoinSubqueriesPreprocessor(context).visit(query);
/// Optimizes logical expressions.
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();

View File

@ -57,7 +57,8 @@ std::shared_ptr<TSystemLog> createSystemLog(
else
{
String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)");
engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024";
engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"
"SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default
}
size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS);

View File

@ -101,9 +101,9 @@ namespace
using Conjunction = ColumnsWithTypeAndName;
using Disjunction = std::vector<Conjunction>;
Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr)
Disjunction analyzeEquals(const ASTIdentifier * identifier, const Field & value, const ExpressionActionsPtr & expr)
{
if (!identifier || !literal)
if (!identifier || value.isNull())
{
return {};
}
@ -116,10 +116,10 @@ namespace
if (name == identifier->name)
{
ColumnWithTypeAndName column;
Field value = convertFieldToType(literal->value, *type);
if (!literal->value.isNull() && value.isNull())
Field converted = convertFieldToType(value, *type);
if (converted.isNull())
return {};
column.column = type->createColumnConst(1, value);
column.column = type->createColumnConst(1, converted);
column.name = name;
column.type = type;
return {{std::move(column)}};
@ -129,6 +129,16 @@ namespace
return {};
}
Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr)
{
if (!identifier || !literal)
{
return {};
}
return analyzeEquals(identifier, literal->value, expr);
}
Disjunction andDNF(const Disjunction & left, const Disjunction & right)
{
if (left.empty())
@ -174,33 +184,44 @@ namespace
const auto * left = fn->arguments->children.front().get();
const auto * right = fn->arguments->children.back().get();
const auto * identifier = left->as<ASTIdentifier>();
const auto * inner_fn = right->as<ASTFunction>();
if (!inner_fn)
{
return {};
}
const auto * tuple = inner_fn->children.front()->as<ASTExpressionList>();
if (!tuple)
{
return {};
}
Disjunction result;
for (const auto & child : tuple->children)
if (const auto * tuple_func = right->as<ASTFunction>(); tuple_func && tuple_func->name == "tuple")
{
const auto * literal = child->as<ASTLiteral>();
const auto dnf = analyzeEquals(identifier, literal, expr);
if (dnf.empty())
const auto * tuple_elements = tuple_func->children.front()->as<ASTExpressionList>();
for (const auto & child : tuple_elements->children)
{
return {};
}
const auto * literal = child->as<ASTLiteral>();
const auto dnf = analyzeEquals(identifier, literal, expr);
result.insert(result.end(), dnf.begin(), dnf.end());
if (dnf.empty())
{
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
}
}
else if (const auto * tuple_literal = right->as<ASTLiteral>();
tuple_literal && tuple_literal->value.getType() == Field::Types::Tuple)
{
const auto & tuple = tuple_literal->value.get<const Tuple &>();
for (const auto & child : tuple)
{
const auto dnf = analyzeEquals(identifier, child, expr);
if (dnf.empty())
{
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
}
}
else
{
return {};
}
return result;

View File

@ -41,6 +41,7 @@
#include <Processors/Formats/IOutputFormat.h>
#include <Parsers/ASTWatchQuery.h>
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;

View File

@ -38,7 +38,7 @@ void executeQuery(
/// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections)
/// must be done separately.
BlockIO executeQuery(
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed.
@ -48,7 +48,7 @@ BlockIO executeQuery(
QueryPipeline executeQueryWithProcessors(
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed.

View File

@ -1181,7 +1181,8 @@ TestResult check(const TestEntry & entry)
try
{
DB::InJoinSubqueriesPreprocessor(context, std::make_unique<CheckShardsAndTablesMock>()).visit(ast_input);
DB::InJoinSubqueriesPreprocessor::SubqueryTables renamed;
DB::InJoinSubqueriesPreprocessor(context, renamed, std::make_unique<CheckShardsAndTablesMock>()).visit(ast_input);
}
catch (const DB::Exception & ex)
{

View File

@ -16,17 +16,21 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const
void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const
{
/// Special case for very large arrays. Instead of listing all elements, will use hash of them.
/// 100 - just arbitrary value.
constexpr auto min_elements_for_hashing = 100;
/// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them.
/// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.)
if (value.getType() == Field::Types::Array
&& value.get<const Array &>().size() > 100) /// 100 - just arbitrary value.
auto type = value.getType();
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing)
|| (type == Field::Types::Tuple && value.get<const Tuple &>().size() > min_elements_for_hashing))
{
SipHash hash;
applyVisitor(FieldVisitorHash(hash), value);
UInt64 low, high;
hash.get128(low, high);
writeCString("__array_", ostr);
writeCString(type == Field::Types::Array ? "__array_" : "__tuple_", ostr);
writeText(low, ostr);
ostr.write('_');
writeText(high, ostr);

View File

@ -1026,27 +1026,31 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
return true;
}
bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
template <typename Collection>
bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::OpeningSquareBracket)
if (pos->type != opening_bracket)
return false;
Pos literal_begin = pos;
Array arr;
Collection arr;
ParserLiteral literal_p;
++pos;
while (pos.isValid())
{
if (!arr.empty())
{
if (pos->type == TokenType::ClosingSquareBracket)
if (pos->type == closing_bracket)
{
auto literal = std::make_shared<ASTLiteral>(arr);
std::shared_ptr<ASTLiteral> literal;
/// Parse one-element tuples (e.g. (1)) later as single values for backward compatibility.
if (std::is_same_v<Collection, Tuple> && arr.size() == 1)
return false;
literal = std::make_shared<ASTLiteral>(arr);
literal->begin = literal_begin;
literal->end = ++pos;
node = literal;
@ -1058,7 +1062,8 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
else
{
expected.add(pos, "comma or closing square bracket");
String message = String("comma or ") + getTokenName(closing_bracket);
expected.add(pos, message.c_str());
return false;
}
}
@ -1070,7 +1075,7 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
arr.push_back(literal_node->as<ASTLiteral &>().value);
}
expected.add(pos, "closing square bracket");
expected.add(pos, getTokenName(closing_bracket));
return false;
}
@ -1271,6 +1276,7 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserSubquery().parse(pos, node, expected)
|| ParserTupleOfLiterals().parse(pos, node, expected)
|| ParserParenthesisExpression().parse(pos, node, expected)
|| ParserArrayOfLiterals().parse(pos, node, expected)
|| ParserArray().parse(pos, node, expected)

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/Field.h>
#include <Parsers/IParserBase.h>
@ -223,17 +224,50 @@ protected:
};
/** An array of literals.
* Arrays can also be parsed as an application of [] operator.
* But parsing the whole array as a whole constant seriously speeds up the analysis of expressions in the case of very large arrays.
* We try to parse the array as an array of literals first (fast path),
* and if it did not work out (when the array consists of complex expressions) - parse as an application of [] operator (slow path).
/** An array or tuple of literals.
* Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function.
* But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection.
* We try to parse the array or tuple as a collection of literals first (fast path),
* and if it did not work out (when the collection consists of complex expressions) -
* parse as an application of [] operator or 'tuple' function (slow path).
*/
template <typename Collection>
class ParserCollectionOfLiterals : public IParserBase
{
public:
ParserCollectionOfLiterals(TokenType opening_bracket_, TokenType closing_bracket_)
: opening_bracket(opening_bracket_), closing_bracket(closing_bracket_) {}
protected:
const char * getName() const override { return "collection of literals"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
TokenType opening_bracket;
TokenType closing_bracket;
};
/// A tuple of literals with same type.
class ParserTupleOfLiterals : public IParserBase
{
public:
ParserCollectionOfLiterals<Tuple> tuple_parser{TokenType::OpeningRoundBracket, TokenType::ClosingRoundBracket};
protected:
const char * getName() const override { return "tuple"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return tuple_parser.parse(pos, node, expected);
}
};
class ParserArrayOfLiterals : public IParserBase
{
public:
ParserCollectionOfLiterals<Array> array_parser{TokenType::OpeningSquareBracket, TokenType::ClosingSquareBracket};
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return array_parser.parse(pos, node, expected);
}
};

View File

@ -4,6 +4,7 @@
#include <Parsers/parseUserName.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
@ -168,34 +169,38 @@ namespace
else if (ParserKeyword{"NAME REGEXP"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
return false;
new_hosts.addNameRegexp(ast->as<const ASTLiteral &>().value.safeGet<String>());
for (const auto & name_regexp_ast : ast->children)
new_hosts.addNameRegexp(name_regexp_ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"NAME"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
return false;
new_hosts.addName(ast->as<const ASTLiteral &>().value.safeGet<String>());
for (const auto & name_ast : ast->children)
new_hosts.addName(name_ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"IP"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
return false;
new_hosts.addSubnet(ast->as<const ASTLiteral &>().value.safeGet<String>());
for (const auto & subnet_ast : ast->children)
new_hosts.addSubnet(subnet_ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"LIKE"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
return false;
new_hosts.addLikePattern(ast->as<const ASTLiteral &>().value.safeGet<String>());
for (const auto & pattern_ast : ast->children)
new_hosts.addLikePattern(pattern_ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else
return false;

View File

@ -38,14 +38,34 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
String fields_str;
const auto * tuple_ast = value->as<ASTFunction>();
bool surrounded_by_parens = false;
if (tuple_ast && tuple_ast->name == "tuple")
{
surrounded_by_parens = true;
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
if (arguments_ast)
fields_count = arguments_ast->children.size();
else
fields_count = 0;
}
else if (auto literal = value->as<ASTLiteral>())
{
if (literal->value.getType() == Field::Types::Tuple)
{
surrounded_by_parens = true;
fields_count = literal->value.get<Tuple &>().size();
}
else
{
fields_count = 1;
fields_str = String(begin->begin, pos->begin - begin->begin);
}
}
else
return false;
if (surrounded_by_parens)
{
Pos left_paren = begin;
Pos right_paren = pos;
@ -61,13 +81,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
fields_str = String(left_paren->end, right_paren->begin - left_paren->end);
}
else if (value->as<ASTLiteral>())
{
fields_count = 1;
fields_str = String(begin->begin, pos->begin - begin->begin);
}
else
return false;
partition->value = value;
partition->children.push_back(value);

View File

@ -30,13 +30,27 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
}
struct SpecialParserType
{
bool is_array = false;
bool is_nullable = false;
Field::Types::Which nested_type = Field::Types::Which::String;
SpecialParserType() = default;
explicit SpecialParserType(Field::Types::Which main_type_) : main_type(main_type_) {}
bool useDefaultParser() const { return nested_type == Field::Types::Which::String; }
Field::Types::Which main_type = Field::Types::String;
bool is_nullable = false;
bool is_array = false;
bool is_tuple = false;
/// Type and nullability
std::vector<std::pair<Field::Types::Which, bool>> nested_types;
bool useDefaultParser() const
{
return main_type == Field::Types::String || (!nested_types.empty()
&& std::all_of(
nested_types.begin(),
nested_types.end(),
[](const auto & type) { return type.first == Field::Types::String; }));
}
};
struct LiteralInfo
@ -54,6 +68,54 @@ struct LiteralInfo
SpecialParserType special_parser;
};
static void fillLiteralInfo(DataTypes & nested_types, LiteralInfo & info)
{
size_t elements_num = nested_types.size();
info.special_parser.nested_types.reserve(elements_num);
for (auto nested_type : nested_types)
{
/// It can be Array(Nullable(nested_type)) or Tuple(..., Nullable(nested_type), ...)
bool is_nullable = false;
if (auto nullable = dynamic_cast<const DataTypeNullable *>(nested_type.get()))
{
nested_type = nullable->getNestedType();
is_nullable = true;
}
WhichDataType type_info{nested_type};
Field::Types::Which field_type;
/// Promote integers to 64 bit types
if (type_info.isNativeUInt())
{
nested_type = std::make_shared<DataTypeUInt64>();
field_type = Field::Types::UInt64;
}
else if (type_info.isNativeInt())
{
nested_type = std::make_shared<DataTypeInt64>();
field_type = Field::Types::Int64;
}
else if (type_info.isFloat64())
{
field_type = Field::Types::Float64;
}
else if (type_info.isString())
{
field_type = Field::Types::String;
}
else
throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug",
ErrorCodes::LOGICAL_ERROR);
if (is_nullable)
nested_type = std::make_shared<DataTypeNullable>(nested_type);
info.special_parser.nested_types.emplace_back(field_type, is_nullable);
}
}
/// Extracts ASTLiterals from expression, replaces them with ASTIdentifiers where needed
/// and deduces data types for dummy columns by field type of literal
class ReplaceLiteralsVisitor
@ -137,7 +199,7 @@ private:
/// We have to use ParserNumber instead of type->deserializeAsTextQuoted() for arithmetic types
/// to check actual type of literal and avoid possible overflow and precision issues.
info.special_parser = SpecialParserType{false, false, field_type};
info.special_parser = SpecialParserType(field_type);
/// Do not use 8, 16 and 32 bit types, so template will match all integers
if (field_type == Field::Types::UInt64)
@ -152,47 +214,17 @@ private:
{
info.special_parser.is_array = true;
info.type = applyVisitor(FieldToDataType(), info.literal->value);
auto nested_type = assert_cast<const DataTypeArray &>(*info.type).getNestedType();
/// It can be Array(Nullable(nested_type))
bool array_of_nullable = false;
if (auto nullable = dynamic_cast<const DataTypeNullable *>(nested_type.get()))
{
nested_type = nullable->getNestedType();
array_of_nullable = true;
}
WhichDataType type_info{nested_type};
/// Promote integers to 64 bit types
if (type_info.isNativeUInt())
{
nested_type = std::make_shared<DataTypeUInt64>();
info.special_parser.nested_type = Field::Types::UInt64;
}
else if (type_info.isNativeInt())
{
nested_type = std::make_shared<DataTypeInt64>();
info.special_parser.nested_type = Field::Types::Int64;
}
else if (type_info.isFloat64())
{
info.special_parser.nested_type = Field::Types::Float64;
}
else if (type_info.isString())
{
info.special_parser.nested_type = Field::Types::String;
}
else
throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug",
ErrorCodes::LOGICAL_ERROR);
if (array_of_nullable)
{
nested_type = std::make_shared<DataTypeNullable>(nested_type);
info.special_parser.is_nullable = true;
}
info.type = std::make_shared<DataTypeArray>(nested_type);
DataTypes nested_types = { assert_cast<const DataTypeArray &>(*info.type).getNestedType() };
fillLiteralInfo(nested_types, info);
info.type = std::make_shared<DataTypeArray>(nested_types[0]);
}
else if (field_type == Field::Types::Tuple)
{
info.special_parser.is_tuple = true;
info.type = applyVisitor(FieldToDataType(), info.literal->value);
auto nested_types = assert_cast<const DataTypeTuple &>(*info.type).getElements();
fillLiteralInfo(nested_types, info);
info.type = std::make_shared<DataTypeTuple>(nested_types);
}
else
throw Exception(String("Unexpected literal type ") + info.literal->value.getTypeName() + ". It's a bug",
@ -404,36 +436,50 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
/// If literal does not fit entirely in the buffer, parsing error will happen.
/// However, it's possible to deduce new template (or use template from cache) after error like it was template mismatch.
if (type_info.is_array)
if (type_info.is_array || type_info.is_tuple)
{
/// TODO faster way to check types without using Parsers
ParserArrayOfLiterals parser_array;
ParserTupleOfLiterals parser_tuple;
Tokens tokens_number(istr.position(), istr.buffer().end());
IParser::Pos iterator(tokens_number, settings.max_parser_depth);
Expected expected;
ASTPtr ast;
if (!parser_array.parse(iterator, ast, expected))
if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected))
return false;
istr.position() = const_cast<char *>(iterator->begin);
const Field & array = ast->as<ASTLiteral &>().value;
auto array_type = applyVisitor(FieldToDataType(), array);
auto nested_type = assert_cast<const DataTypeArray &>(*array_type).getNestedType();
if (type_info.is_nullable)
if (auto nullable = dynamic_cast<const DataTypeNullable *>(nested_type.get()))
nested_type = nullable->getNestedType();
const Field & collection = ast->as<ASTLiteral &>().value;
auto collection_type = applyVisitor(FieldToDataType(), collection);
WhichDataType nested_type_info(nested_type);
if ((nested_type_info.isNativeUInt() && type_info.nested_type == Type::UInt64) ||
(nested_type_info.isNativeInt() && type_info.nested_type == Type::Int64) ||
(nested_type_info.isFloat64() && type_info.nested_type == Type::Float64))
DataTypes nested_types;
if (type_info.is_array)
nested_types = { assert_cast<const DataTypeArray &>(*collection_type).getNestedType() };
else
nested_types = assert_cast<const DataTypeTuple &>(*collection_type).getElements();
for (size_t i = 0; i < nested_types.size(); ++i)
{
Field array_same_types = convertFieldToType(array, *complex_type, nullptr);
columns[column_idx]->insert(array_same_types);
return true;
const auto & [nested_field_type, is_nullable] = type_info.nested_types[i];
if (is_nullable)
if (auto nullable = dynamic_cast<const DataTypeNullable *>(nested_types[i].get()))
nested_types[i] = nullable->getNestedType();
WhichDataType nested_type_info(nested_types[i]);
bool are_types_compatible =
(nested_type_info.isNativeUInt() && nested_field_type == Type::UInt64) ||
(nested_type_info.isNativeInt() && nested_field_type == Type::Int64) ||
(nested_type_info.isFloat64() && nested_field_type == Type::Float64);
if (!are_types_compatible)
return false;
}
return false;
Field array_same_types = convertFieldToType(collection, *complex_type, nullptr);
columns[column_idx]->insert(array_same_types);
return true;
}
else
{
@ -470,14 +516,14 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
if (pos_integer == pos_double && errno != ERANGE && (!negative || uint_value <= (1ULL << 63)))
{
istr.position() += pos_integer - buf;
if (negative && type_info.nested_type == Type::Int64)
if (negative && type_info.main_type == Type::Int64)
number = static_cast<Int64>(-uint_value);
else if (!negative && type_info.nested_type == Type::UInt64)
else if (!negative && type_info.main_type == Type::UInt64)
number = uint_value;
else
return false;
}
else if (type_info.nested_type == Type::Float64)
else if (type_info.main_type == Type::Float64)
{
istr.position() += pos_double - buf;
number = float_value;

View File

@ -0,0 +1,195 @@
#include <stdlib.h>
#include <common/find_symbols.h>
#include <Processors/Formats/Impl/RegexpRowInputFormat.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int BAD_ARGUMENTS;
}
RegexpRowInputFormat::RegexpRowInputFormat(
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, in_, std::move(params_)), buf(in_), format_settings(format_settings_), regexp(format_settings_.regexp.regexp)
{
size_t fields_count = regexp.NumberOfCapturingGroups();
matched_fields.resize(fields_count);
re2_arguments.resize(fields_count);
re2_arguments_ptrs.resize(fields_count);
for (size_t i = 0; i != fields_count; ++i)
{
// Bind an argument to a matched field.
re2_arguments[i] = &matched_fields[i];
// Save pointer to argument.
re2_arguments_ptrs[i] = &re2_arguments[i];
}
field_format = stringToFormat(format_settings_.regexp.escaping_rule);
}
RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const String & format)
{
if (format == "Escaped")
return ColumnFormat::Escaped;
if (format == "Quoted")
return ColumnFormat::Quoted;
if (format == "CSV")
return ColumnFormat::Csv;
if (format == "JSON")
return ColumnFormat::Json;
throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS);
}
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
{
const auto & type = getPort().getHeader().getByPosition(index).type;
bool parse_as_nullable = format_settings.null_as_default && !type->isNullable();
bool read = true;
ReadBuffer field_buf(const_cast<char *>(matched_fields[index].data()), matched_fields[index].size(), 0);
try
{
switch (field_format)
{
case ColumnFormat::Escaped:
if (parse_as_nullable)
read = DataTypeNullable::deserializeTextEscaped(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsTextEscaped(*columns[index], field_buf, format_settings);
break;
case ColumnFormat::Quoted:
if (parse_as_nullable)
read = DataTypeNullable::deserializeTextQuoted(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsTextQuoted(*columns[index], field_buf, format_settings);
break;
case ColumnFormat::Csv:
if (parse_as_nullable)
read = DataTypeNullable::deserializeTextCSV(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsTextCSV(*columns[index], field_buf, format_settings);
break;
case ColumnFormat::Json:
if (parse_as_nullable)
read = DataTypeNullable::deserializeTextJSON(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsTextJSON(*columns[index], field_buf, format_settings);
break;
default:
break;
}
}
catch (Exception & e)
{
e.addMessage("(while read the value of column " + getPort().getHeader().getByPosition(index).name + ")");
throw;
}
return read;
}
void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext)
{
if (matched_fields.size() != columns.size())
throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA);
ext.read_columns.assign(columns.size(), false);
for (size_t columns_index = 0; columns_index < columns.size(); ++columns_index)
{
ext.read_columns[columns_index] = readField(columns_index, columns);
}
}
bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
{
if (buf.eof())
return false;
PeekableReadBufferCheckpoint checkpoint{buf};
size_t line_size = 0;
do
{
char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end());
line_size += pos - buf.position();
buf.position() = pos;
} while (buf.position() == buf.buffer().end() && !buf.eof());
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
bool read_line = true;
if (!match)
{
if (!format_settings.regexp.skip_unmatched)
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
read_line = false;
}
if (read_line)
readFieldsFromMatch(columns, ext);
buf.position() += line_size;
checkChar('\r', buf);
if (!buf.eof() && !checkChar('\n', buf))
throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA);
return true;
}
void registerInputFormatProcessorRegexp(FormatFactory & factory)
{
factory.registerInputFormatProcessor("Regexp", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<RegexpRowInputFormat>(buf, sample, std::move(params), settings);
});
}
static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
char * pos = in.position();
bool need_more_data = true;
while (loadAtPosition(in, memory, pos) && need_more_data)
{
pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
// Support DOS-style newline ("\r\n")
if (*pos == '\r')
{
++pos;
if (pos == in.buffer().end())
loadAtPosition(in, memory, pos);
}
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
need_more_data = false;
++pos;
}
saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos);
}
void registerFileSegmentationEngineRegexp(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl);
}
}

View File

@ -0,0 +1,53 @@
#pragma once
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <string>
#include <vector>
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
#include <IO/PeekableReadBuffer.h>
#include <Formats/ParsedTemplateFormatString.h>
namespace DB
{
class ReadBuffer;
/// Regexp input format.
/// This format applies regular expression from format_regexp setting for every line of file
/// (the lines must be separated by newline character ('\n') or DOS-style newline ("\r\n")).
/// Every matched subpattern will be parsed with the method of corresponding data type
/// (according to format_regexp_escaping_rule setting). If the regexp did not match the line,
/// if format_regexp_skip_unmatched is 1, the line is silently skipped, if the setting is 0, exception will be thrown.
class RegexpRowInputFormat : public IRowInputFormat
{
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
public:
RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_);
String getName() const override { return "RegexpRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
private:
bool readField(size_t index, MutableColumns & columns);
void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext);
static ColumnFormat stringToFormat(const String & format);
PeekableReadBuffer buf;
const FormatSettings format_settings;
ColumnFormat field_format;
RE2 regexp;
// The vector of fields extracted from line using regexp.
std::vector<re2::StringPiece> matched_fields;
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
std::vector<RE2::Arg> re2_arguments;
std::vector<RE2::Arg *> re2_arguments_ptrs;
};
}

View File

@ -314,11 +314,9 @@ bool IStorage::isVirtualColumn(const String & column_name) const
return getColumns().get(column_name).is_virtual;
}
TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id)
TableStructureReadLockHolder IStorage::lockStructureForShare(const String & query_id)
{
TableStructureReadLockHolder result;
if (will_add_new_data)
result.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Read, query_id);
result.structure_lock = structure_lock->getLock(RWLockImpl::Read, query_id);
if (is_dropped)
@ -326,43 +324,32 @@ TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_d
return result;
}
TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_id)
TableStructureWriteLockHolder IStorage::lockAlterIntention()
{
TableStructureWriteLockHolder result;
result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id);
result.alter_lock = std::unique_lock(alter_lock);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
return result;
}
void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id)
{
if (!lock_holder.alter_intention_lock)
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id);
}
void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id)
{
if (!lock_holder.alter_intention_lock)
if (!lock_holder.alter_lock)
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
if (!lock_holder.new_data_structure_lock)
lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id);
lock_holder.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id);
}
TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
{
TableStructureWriteLockHolder result;
result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id);
result.alter_lock = std::unique_lock(alter_lock);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
result.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id);
result.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id);
return result;

View File

@ -203,16 +203,11 @@ public:
/// Acquire this lock if you need the table structure to remain constant during the execution of
/// the query. If will_add_new_data is true, this means that the query will add new data to the table
/// (INSERT or a parts merge).
TableStructureReadLockHolder lockStructureForShare(bool will_add_new_data, const String & query_id);
TableStructureReadLockHolder lockStructureForShare(const String & query_id);
/// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you
/// can modify the table structure. It can later be upgraded to the exclusive lock.
TableStructureWriteLockHolder lockAlterIntention(const String & query_id);
/// Upgrade alter intention lock and make sure that no new data is inserted into the table.
/// This is used by the ALTER MODIFY of the MergeTree storage to consistently determine
/// the set of parts that needs to be altered.
void lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id);
TableStructureWriteLockHolder lockAlterIntention();
/// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries
/// to ensure that no other query uses the table structure and it can be safely changed.
@ -224,8 +219,12 @@ public:
/** Returns stage to which query is going to be processed in read() function.
* (Normally, the function only reads the columns from the list, but in other cases,
* for example, the request can be partially processed on a remote server.)
*
* SelectQueryInfo is required since the stage can depends on the query
* (see Distributed() engine and optimize_skip_unused_shards).
*/
virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &) const { return QueryProcessingStage::FetchColumns; }
QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const { return getQueryProcessingStage(context, {}); }
virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const { return QueryProcessingStage::FetchColumns; }
/** Watch live changes to the table.
* Accepts a list of columns to read, as well as a description of the query,
@ -465,25 +464,38 @@ public:
/// Returns storage policy if storage supports it
virtual StoragePolicyPtr getStoragePolicy() const { return {}; }
/** If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it.
*/
/// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it.
/// Used for:
/// - Simple count() opimization
/// - For total_rows column in system.tables
///
/// Does takes underlying Storage (if any) into account.
virtual std::optional<UInt64> totalRows() const
{
return {};
}
/// If it is possible to quickly determine exact number of bytes for the table on storage:
/// - memory (approximated)
/// - disk (compressed)
///
/// Used for:
/// - For total_bytes column in system.tables
//
/// Does not takes underlying Storage (if any) into account
/// (since for Buffer we still need to know how much bytes it uses).
virtual std::optional<UInt64> totalBytes() const
{
return {};
}
private:
/// You always need to take the next three locks in this order.
/// If you hold this lock exclusively, you can be sure that no other structure modifying queries
/// (e.g. ALTER, DROP) are concurrently executing. But queries that only read table structure
/// (e.g. SELECT, INSERT) can continue to execute.
mutable RWLock alter_intention_lock = RWLockImpl::create();
/// It is taken for share for the entire INSERT query and the entire merge of the parts (for MergeTree).
/// ALTER COLUMN queries acquire an exclusive lock to ensure that no new parts with the old structure
/// are added to the table and thus the set of parts to modify doesn't change.
mutable RWLock new_data_structure_lock = RWLockImpl::create();
mutable std::mutex alter_lock;
/// Lock for the table column structure (names, types, etc.) and data path.
/// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP)

View File

@ -26,7 +26,7 @@ public:
return std::make_shared<StorageBlocks>(table_id, columns, std::move(pipes), to_stage);
}
std::string getName() const override { return "Blocks"; }
QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; }
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const override { return to_stage; }
Pipes read(
const Names & /*column_names*/,

View File

@ -518,14 +518,11 @@ void StorageLiveView::drop()
condition.notify_all();
}
void StorageLiveView::refresh(const Context & context)
void StorageLiveView::refresh()
{
auto alter_lock = lockAlterIntention(context.getCurrentQueryId());
{
std::lock_guard lock(mutex);
if (getNewBlocks())
condition.notify_all();
}
std::lock_guard lock(mutex);
if (getNewBlocks())
condition.notify_all();
}
Pipes StorageLiveView::read(

View File

@ -123,7 +123,7 @@ public:
void startup() override;
void shutdown() override;
void refresh(const Context & context);
void refresh();
Pipes read(
const Names & column_names,

View File

@ -85,7 +85,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
try
{
auto storage_lock = data.lockStructureForShare(false, RWLockImpl::NO_QUERY);
auto storage_lock = data.lockStructureForShare(RWLockImpl::NO_QUERY);
MergeTreeData::DataPartPtr part = findPart(part_name);

View File

@ -406,8 +406,8 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
loadColumns(require_columns_checksums);
loadChecksums(require_columns_checksums);
calculateColumnsSizesOnDisk();
loadIndexGranularity();
calculateColumnsSizesOnDisk();
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
loadRowsCount(); /// Must be called after loadIndex() as it uses the value of `index_granularity`.
loadPartitionAndMinMaxIndex();

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