mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into hedged-requests
This commit is contained in:
commit
153bfbfc28
@ -322,8 +322,14 @@ public:
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return (UInt32(t) / 60) % 60;
|
||||
|
||||
UInt32 date = find(t).date;
|
||||
return (UInt32(t) - date) / 60 % 60;
|
||||
/// To consider the DST changing situation within this day.
|
||||
/// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account
|
||||
DayNum index = findIndex(t);
|
||||
UInt32 res = t - lut[index].date;
|
||||
if (lut[index].amount_of_offset_change != 0 && t >= lut[index].date + lut[index].time_at_offset_change)
|
||||
res += lut[index].amount_of_offset_change;
|
||||
|
||||
return res / 60 % 60;
|
||||
}
|
||||
|
||||
inline time_t toStartOfMinute(time_t t) const { return t / 60 * 60; }
|
||||
|
@ -26,6 +26,15 @@ struct ConnectionFailed : public Exception
|
||||
};
|
||||
|
||||
|
||||
/// Connection to MySQL server was lost
|
||||
struct ConnectionLost : public Exception
|
||||
{
|
||||
ConnectionLost(const std::string & msg, int code = 0) : Exception(msg, code) {}
|
||||
const char * name() const throw() override { return "mysqlxx::ConnectionLost"; }
|
||||
const char * className() const throw() override { return "mysqlxx::ConnectionLost"; }
|
||||
};
|
||||
|
||||
|
||||
/// Erroneous query.
|
||||
struct BadQuery : public Exception
|
||||
{
|
||||
|
@ -10,7 +10,6 @@
|
||||
|
||||
#include <common/sleep.h>
|
||||
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
|
||||
|
||||
@ -41,7 +40,9 @@ void Pool::Entry::decrementRefCount()
|
||||
Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & config_name,
|
||||
unsigned default_connections_, unsigned max_connections_,
|
||||
const char * parent_config_name_)
|
||||
: default_connections(default_connections_), max_connections(max_connections_)
|
||||
: logger(Poco::Logger::get("mysqlxx::Pool"))
|
||||
, default_connections(default_connections_)
|
||||
, max_connections(max_connections_)
|
||||
{
|
||||
server = cfg.getString(config_name + ".host");
|
||||
|
||||
@ -130,20 +131,30 @@ Pool::Entry Pool::get()
|
||||
initialize();
|
||||
for (;;)
|
||||
{
|
||||
logger.trace("(%s): Iterating through existing MySQL connections", getDescription());
|
||||
|
||||
for (auto & connection : connections)
|
||||
{
|
||||
if (connection->ref_count == 0)
|
||||
return Entry(connection, this);
|
||||
}
|
||||
|
||||
logger.trace("(%s): Trying to allocate a new connection.", getDescription());
|
||||
if (connections.size() < static_cast<size_t>(max_connections))
|
||||
{
|
||||
Connection * conn = allocConnection();
|
||||
if (conn)
|
||||
return Entry(conn, this);
|
||||
|
||||
logger.trace("(%s): Unable to create a new connection: Allocation failed.", getDescription());
|
||||
}
|
||||
else
|
||||
{
|
||||
logger.trace("(%s): Unable to create a new connection: Max number of connections has been reached.", getDescription());
|
||||
}
|
||||
|
||||
lock.unlock();
|
||||
logger.trace("(%s): Sleeping for %d seconds.", getDescription(), MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL);
|
||||
sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL);
|
||||
lock.lock();
|
||||
}
|
||||
@ -167,8 +178,7 @@ Pool::Entry Pool::tryGet()
|
||||
if (res.tryForceConnected()) /// Tries to reestablish connection as well
|
||||
return res;
|
||||
|
||||
auto & logger = Poco::Util::Application::instance().logger();
|
||||
logger.information("Idle connection to mysql server cannot be recovered, dropping it.");
|
||||
logger.debug("(%s): Idle connection to MySQL server cannot be recovered, dropping it.", getDescription());
|
||||
|
||||
/// This one is disconnected, cannot be reestablished and so needs to be disposed of.
|
||||
connection_it = connections.erase(connection_it);
|
||||
@ -191,6 +201,8 @@ Pool::Entry Pool::tryGet()
|
||||
|
||||
void Pool::removeConnection(Connection* connection)
|
||||
{
|
||||
logger.trace("(%s): Removing connection.", getDescription());
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (connection)
|
||||
{
|
||||
@ -215,8 +227,6 @@ void Pool::Entry::forceConnected() const
|
||||
if (data == nullptr)
|
||||
throw Poco::RuntimeException("Tried to access NULL database connection.");
|
||||
|
||||
Poco::Util::Application & app = Poco::Util::Application::instance();
|
||||
|
||||
bool first = true;
|
||||
while (!tryForceConnected())
|
||||
{
|
||||
@ -225,7 +235,7 @@ void Pool::Entry::forceConnected() const
|
||||
else
|
||||
sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL);
|
||||
|
||||
app.logger().information("MYSQL: Reconnecting to " + pool->description);
|
||||
pool->logger.debug("Entry: Reconnecting to MySQL server %s", pool->description);
|
||||
data->conn.connect(
|
||||
pool->db.c_str(),
|
||||
pool->server.c_str(),
|
||||
@ -248,18 +258,22 @@ bool Pool::Entry::tryForceConnected() const
|
||||
{
|
||||
auto * const mysql_driver = data->conn.getDriver();
|
||||
const auto prev_connection_id = mysql_thread_id(mysql_driver);
|
||||
|
||||
pool->logger.trace("Entry(connection %lu): sending PING to check if it is alive.", prev_connection_id);
|
||||
if (data->conn.ping()) /// Attempts to reestablish lost connection
|
||||
{
|
||||
const auto current_connection_id = mysql_thread_id(mysql_driver);
|
||||
if (prev_connection_id != current_connection_id)
|
||||
{
|
||||
auto & logger = Poco::Util::Application::instance().logger();
|
||||
logger.information("Reconnected to mysql server. Connection id changed: %lu -> %lu",
|
||||
prev_connection_id, current_connection_id);
|
||||
pool->logger.debug("Entry(connection %lu): Reconnected to MySQL server. Connection id changed: %lu -> %lu",
|
||||
current_connection_id, prev_connection_id, current_connection_id);
|
||||
}
|
||||
|
||||
pool->logger.trace("Entry(connection %lu): PING ok.", current_connection_id);
|
||||
return true;
|
||||
}
|
||||
|
||||
pool->logger.trace("Entry(connection %lu): PING failed.", prev_connection_id);
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -280,15 +294,13 @@ void Pool::initialize()
|
||||
|
||||
Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time)
|
||||
{
|
||||
Poco::Util::Application & app = Poco::Util::Application::instance();
|
||||
|
||||
std::unique_ptr<Connection> conn(new Connection);
|
||||
std::unique_ptr<Connection> conn_ptr{new Connection};
|
||||
|
||||
try
|
||||
{
|
||||
app.logger().information("MYSQL: Connecting to " + description);
|
||||
logger.debug("Connecting to %s", description);
|
||||
|
||||
conn->conn.connect(
|
||||
conn_ptr->conn.connect(
|
||||
db.c_str(),
|
||||
server.c_str(),
|
||||
user.c_str(),
|
||||
@ -305,25 +317,24 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time)
|
||||
}
|
||||
catch (mysqlxx::ConnectionFailed & e)
|
||||
{
|
||||
logger.error(e.what());
|
||||
|
||||
if ((!was_successful && !dont_throw_if_failed_first_time)
|
||||
|| e.errnum() == ER_ACCESS_DENIED_ERROR
|
||||
|| e.errnum() == ER_DBACCESS_DENIED_ERROR
|
||||
|| e.errnum() == ER_BAD_DB_ERROR)
|
||||
{
|
||||
app.logger().error(e.what());
|
||||
throw;
|
||||
}
|
||||
else
|
||||
{
|
||||
app.logger().error(e.what());
|
||||
return nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
connections.push_back(conn_ptr.get());
|
||||
was_successful = true;
|
||||
auto * connection = conn.release();
|
||||
connections.push_back(connection);
|
||||
return connection;
|
||||
return conn_ptr.release();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <atomic>
|
||||
|
||||
#include <Poco/Exception.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
#include <mysqlxx/Connection.h>
|
||||
|
||||
|
||||
@ -167,13 +169,13 @@ public:
|
||||
unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS,
|
||||
unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE,
|
||||
bool opt_reconnect_ = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT)
|
||||
: default_connections(default_connections_), max_connections(max_connections_),
|
||||
db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_),
|
||||
: logger(Poco::Logger::get("mysqlxx::Pool")), default_connections(default_connections_),
|
||||
max_connections(max_connections_), db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_),
|
||||
connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_),
|
||||
opt_reconnect(opt_reconnect_) {}
|
||||
|
||||
Pool(const Pool & other)
|
||||
: default_connections{other.default_connections},
|
||||
: logger(other.logger), default_connections{other.default_connections},
|
||||
max_connections{other.max_connections},
|
||||
db{other.db}, server{other.server},
|
||||
user{other.user}, password{other.password},
|
||||
@ -203,6 +205,8 @@ public:
|
||||
void removeConnection(Connection * connection);
|
||||
|
||||
protected:
|
||||
Poco::Logger & logger;
|
||||
|
||||
/// Number of MySQL connections which are created at launch.
|
||||
unsigned default_connections;
|
||||
/// Maximum possible number of connections
|
||||
|
@ -1,11 +1,16 @@
|
||||
#if __has_include(<mysql.h>)
|
||||
#include <errmsg.h>
|
||||
#include <mysql.h>
|
||||
#else
|
||||
#include <mysql/errmsg.h>
|
||||
#include <mysql/mysql.h>
|
||||
#endif
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
#include <mysqlxx/Connection.h>
|
||||
#include <mysqlxx/Query.h>
|
||||
#include <mysqlxx/Types.h>
|
||||
|
||||
|
||||
namespace mysqlxx
|
||||
@ -57,8 +62,24 @@ void Query::reset()
|
||||
void Query::executeImpl()
|
||||
{
|
||||
std::string query_string = query_buf.str();
|
||||
if (mysql_real_query(conn->getDriver(), query_string.data(), query_string.size()))
|
||||
throw BadQuery(errorMessage(conn->getDriver()), mysql_errno(conn->getDriver()));
|
||||
|
||||
MYSQL* mysql_driver = conn->getDriver();
|
||||
|
||||
auto & logger = Poco::Logger::get("mysqlxx::Query");
|
||||
logger.trace("Running MySQL query using connection %lu", mysql_thread_id(mysql_driver));
|
||||
if (mysql_real_query(mysql_driver, query_string.data(), query_string.size()))
|
||||
{
|
||||
const auto err_no = mysql_errno(mysql_driver);
|
||||
switch (err_no)
|
||||
{
|
||||
case CR_SERVER_GONE_ERROR:
|
||||
[[fallthrough]];
|
||||
case CR_SERVER_LOST:
|
||||
throw ConnectionLost(errorMessage(mysql_driver), err_no);
|
||||
default:
|
||||
throw BadQuery(errorMessage(mysql_driver), err_no);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
UseQueryResult Query::use()
|
||||
|
2
contrib/brotli
vendored
2
contrib/brotli
vendored
@ -1 +1 @@
|
||||
Subproject commit 5805f99a533a8f8118699c0100d8c102f3605f65
|
||||
Subproject commit 63be8a99401992075c23e99f7c84de1c653e39e2
|
@ -2,6 +2,8 @@ set(BROTLI_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/brotli/c)
|
||||
set(BROTLI_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/brotli/c)
|
||||
|
||||
set(SRCS
|
||||
${BROTLI_SOURCE_DIR}/enc/command.c
|
||||
${BROTLI_SOURCE_DIR}/enc/fast_log.c
|
||||
${BROTLI_SOURCE_DIR}/dec/bit_reader.c
|
||||
${BROTLI_SOURCE_DIR}/dec/state.c
|
||||
${BROTLI_SOURCE_DIR}/dec/huffman.c
|
||||
@ -26,6 +28,9 @@ set(SRCS
|
||||
${BROTLI_SOURCE_DIR}/enc/memory.c
|
||||
${BROTLI_SOURCE_DIR}/common/dictionary.c
|
||||
${BROTLI_SOURCE_DIR}/common/transform.c
|
||||
${BROTLI_SOURCE_DIR}/common/platform.c
|
||||
${BROTLI_SOURCE_DIR}/common/context.c
|
||||
${BROTLI_SOURCE_DIR}/common/constants.c
|
||||
)
|
||||
|
||||
add_library(brotli ${SRCS})
|
||||
|
@ -356,7 +356,6 @@ function run_tests
|
||||
|
||||
# JSON functions
|
||||
01666_blns
|
||||
01674_htm_xml_coarse_parse
|
||||
)
|
||||
|
||||
(time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
|
||||
|
@ -18,7 +18,8 @@ RUN apt-get update \
|
||||
curl \
|
||||
tar \
|
||||
krb5-user \
|
||||
iproute2
|
||||
iproute2 \
|
||||
lsof
|
||||
RUN rm -rf \
|
||||
/var/lib/apt/lists/* \
|
||||
/var/cache/debconf \
|
||||
|
@ -1,11 +1,11 @@
|
||||
version: '2.3'
|
||||
services:
|
||||
zoo1:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
restart: always
|
||||
environment:
|
||||
ZOO_TICK_TIME: 500
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
|
||||
ZOO_MY_ID: 1
|
||||
JVMFLAGS: -Dzookeeper.forceSync=no
|
||||
volumes:
|
||||
@ -16,11 +16,11 @@ services:
|
||||
source: ${ZK_DATA_LOG1:-}
|
||||
target: /datalog
|
||||
zoo2:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
restart: always
|
||||
environment:
|
||||
ZOO_TICK_TIME: 500
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888
|
||||
ZOO_MY_ID: 2
|
||||
JVMFLAGS: -Dzookeeper.forceSync=no
|
||||
volumes:
|
||||
@ -31,11 +31,11 @@ services:
|
||||
source: ${ZK_DATA_LOG2:-}
|
||||
target: /datalog
|
||||
zoo3:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
restart: always
|
||||
environment:
|
||||
ZOO_TICK_TIME: 500
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888
|
||||
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
|
||||
ZOO_MY_ID: 3
|
||||
JVMFLAGS: -Dzookeeper.forceSync=no
|
||||
volumes:
|
||||
|
@ -26,4 +26,4 @@ The name of an additional section can be any, for example, **Usage**.
|
||||
|
||||
- [link](#)
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/data_types/<data-type-name>/) <!--hide-->
|
||||
[Original article](https://clickhouse.tech/docs/en/data-types/<data-type-name>/) <!--hide-->
|
||||
|
@ -38,20 +38,20 @@ SETTINGS
|
||||
|
||||
Required parameters:
|
||||
|
||||
- `kafka_broker_list` – A comma-separated list of brokers (for example, `localhost:9092`).
|
||||
- `kafka_topic_list` – A list of Kafka topics.
|
||||
- `kafka_group_name` – A group of Kafka consumers. Reading margins are tracked for each group separately. If you don’t want messages to be duplicated in the cluster, use the same group name everywhere.
|
||||
- `kafka_format` – Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
|
||||
- `kafka_broker_list` — A comma-separated list of brokers (for example, `localhost:9092`).
|
||||
- `kafka_topic_list` — A list of Kafka topics.
|
||||
- `kafka_group_name` — A group of Kafka consumers. Reading margins are tracked for each group separately. If you don’t want messages to be duplicated in the cluster, use the same group name everywhere.
|
||||
- `kafka_format` — Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
|
||||
|
||||
Optional parameters:
|
||||
|
||||
- `kafka_row_delimiter` – Delimiter character, which ends the message.
|
||||
- `kafka_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
|
||||
- `kafka_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition.
|
||||
- `kafka_max_block_size` - The maximum batch size (in messages) for poll (default: `max_block_size`).
|
||||
- `kafka_skip_broken_messages` – Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data).
|
||||
- `kafka_commit_every_batch` - Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`).
|
||||
- `kafka_thread_per_consumer` - Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise - rows from several consumers squashed to form one block).
|
||||
- `kafka_row_delimiter` — Delimiter character, which ends the message.
|
||||
- `kafka_schema` — Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
|
||||
- `kafka_num_consumers` — The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition.
|
||||
- `kafka_max_block_size` — The maximum batch size (in messages) for poll (default: `max_block_size`).
|
||||
- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data).
|
||||
- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`).
|
||||
- `kafka_thread_per_consumer` — Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block).
|
||||
|
||||
Examples:
|
||||
|
||||
|
133
docs/en/getting-started/example-datasets/cell-towers.md
Normal file
133
docs/en/getting-started/example-datasets/cell-towers.md
Normal file
File diff suppressed because one or more lines are too long
@ -20,5 +20,6 @@ The list of documented datasets:
|
||||
- [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md)
|
||||
- [AMPLab Big Data Benchmark](../../getting-started/example-datasets/amplab-benchmark.md)
|
||||
- [Brown University Benchmark](../../getting-started/example-datasets/brown-benchmark.md)
|
||||
- [Cell Towers](../../getting-started/example-datasets/cell-towers.md)
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets) <!--hide-->
|
||||
|
@ -15,17 +15,9 @@ This dataset can be obtained in two ways:
|
||||
Downloading data:
|
||||
|
||||
``` bash
|
||||
for s in `seq 1987 2018`
|
||||
do
|
||||
for m in `seq 1 12`
|
||||
do
|
||||
wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip
|
||||
done
|
||||
done
|
||||
echo https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip | xargs -P10 wget --no-check-certificate --continue
|
||||
```
|
||||
|
||||
(from https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh )
|
||||
|
||||
Creating a table:
|
||||
|
||||
``` sql
|
||||
@ -145,12 +137,14 @@ ORDER BY (Carrier, FlightDate)
|
||||
SETTINGS index_granularity = 8192;
|
||||
```
|
||||
|
||||
Loading data:
|
||||
Loading data with multiple threads:
|
||||
|
||||
``` bash
|
||||
$ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --input_format_with_names_use_header=0 --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done
|
||||
ls -1 *.zip | xargs -I{} -P $(nproc) bash -c "echo {}; unzip -cq {} '*.csv' | sed 's/\.00//g' | clickhouse-client --input_format_with_names_use_header=0 --query='INSERT INTO ontime FORMAT CSVWithNames'"
|
||||
```
|
||||
|
||||
(if you will have memory shortage or other issues on your server, remove the `-P $(nproc)` part)
|
||||
|
||||
## Download of Prepared Partitions {#download-of-prepared-partitions}
|
||||
|
||||
``` bash
|
||||
|
@ -1956,8 +1956,8 @@ Default value: 16.
|
||||
|
||||
**See Also**
|
||||
|
||||
- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine
|
||||
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine
|
||||
- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine.
|
||||
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine.
|
||||
|
||||
## validate_polygons {#validate_polygons}
|
||||
|
||||
@ -2658,8 +2658,6 @@ Result:
|
||||
|
||||
Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md) behaviour.
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
|
||||
|
||||
## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists}
|
||||
|
||||
Allows to select data from a file engine table without file.
|
||||
@ -2679,3 +2677,16 @@ Possible values:
|
||||
- 1 — Enabled.
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
## allow_experimental_geo_types {#allow-experimental-geo-types}
|
||||
|
||||
Allows working with experimental [geo data types](../../sql-reference/data-types/geo.md).
|
||||
|
||||
Possible values:
|
||||
|
||||
- 0 — Working with geo data types is disabled.
|
||||
- 1 — Working with geo data types is enabled.
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
|
||||
|
@ -20,7 +20,7 @@ System tables:
|
||||
|
||||
Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start.
|
||||
|
||||
Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), crash_log and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one.
|
||||
Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash-log.md) and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one.
|
||||
|
||||
System log tables can be customized by creating a config file with the same name as the table under `/etc/clickhouse-server/config.d/`, or setting corresponding elements in `/etc/clickhouse-server/config.xml`. Elements can be customized are:
|
||||
|
||||
@ -33,7 +33,7 @@ System log tables can be customized by creating a config file with the same name
|
||||
|
||||
An example:
|
||||
|
||||
```
|
||||
```xml
|
||||
<yandex>
|
||||
<query_log>
|
||||
<database>system</database>
|
||||
|
@ -91,6 +91,8 @@ $ clickhouse-local --query "
|
||||
|
||||
Now let’s output memory user for each Unix user:
|
||||
|
||||
Query:
|
||||
|
||||
``` bash
|
||||
$ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \
|
||||
| clickhouse-local --structure "user String, mem Float64" \
|
||||
@ -98,6 +100,8 @@ $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \
|
||||
FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty"
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec.
|
||||
┏━━━━━━━━━━┳━━━━━━━━━━┓
|
||||
|
@ -253,8 +253,8 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `window` — Length of the sliding window. The unit of `window` depends on the timestamp itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`.
|
||||
- `mode` - It is an optional parameter.
|
||||
- `window` — Length of the sliding window. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`.
|
||||
- `mode` - It is an optional argument.
|
||||
- `'strict'` - When the `'strict'` is set, the windowFunnel() applies conditions only for the unique values.
|
||||
|
||||
**Returned value**
|
||||
|
106
docs/en/sql-reference/data-types/geo.md
Normal file
106
docs/en/sql-reference/data-types/geo.md
Normal file
@ -0,0 +1,106 @@
|
||||
---
|
||||
toc_priority: 62
|
||||
toc_title: Geo
|
||||
---
|
||||
|
||||
# Geo Data Types {#geo-data-types}
|
||||
|
||||
Clickhouse supports data types for representing geographical objects — locations, lands, etc.
|
||||
|
||||
!!! warning "Warning"
|
||||
Currently geo data types are an experimental feature. To work with them you must set `allow_experimental_geo_types = 1`.
|
||||
|
||||
**See Also**
|
||||
- [Representing simple geographical features](https://en.wikipedia.org/wiki/GeoJSON).
|
||||
- [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types) setting.
|
||||
|
||||
## Point {#point-data-type}
|
||||
|
||||
`Point` is represented by its X and Y coordinates, stored as a [Tuple](tuple.md)([Float64](float.md), [Float64](float.md)).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_point (p Point) ENGINE = Memory();
|
||||
INSERT INTO geo_point VALUES((10, 10));
|
||||
SELECT p, toTypeName(p) FROM geo_point;
|
||||
```
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─p─────┬─toTypeName(p)─┐
|
||||
│ (10,10) │ Point │
|
||||
└───────┴───────────────┘
|
||||
```
|
||||
|
||||
## Ring {#ring-data-type}
|
||||
|
||||
`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point-data-type)).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_ring (r Ring) ENGINE = Memory();
|
||||
INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
SELECT r, toTypeName(r) FROM geo_ring;
|
||||
```
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─r─────────────────────────────┬─toTypeName(r)─┐
|
||||
│ [(0,0),(10,0),(10,10),(0,10)] │ Ring │
|
||||
└───────────────────────────────┴───────────────┘
|
||||
```
|
||||
|
||||
## Polygon {#polygon-data-type}
|
||||
|
||||
`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring-data-type)). First element of outer array is the outer shape of polygon and all the following elements are holes.
|
||||
|
||||
**Example**
|
||||
|
||||
This is a polygon with one hole:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory();
|
||||
INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]);
|
||||
SELECT pg, toTypeName(pg) FROM geo_polygon;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─pg────────────────────────────────────────────────────────────┬─toTypeName(pg)─┐
|
||||
│ [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] │ Polygon │
|
||||
└───────────────────────────────────────────────────────────────┴────────────────┘
|
||||
```
|
||||
|
||||
## MultiPolygon {#multipolygon-data-type}
|
||||
|
||||
`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon-data-type)).
|
||||
|
||||
**Example**
|
||||
|
||||
This multipolygon consists of two separate polygons — the first one without holes, and the second with one hole:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory();
|
||||
INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]);
|
||||
SELECT mpg, toTypeName(mpg) FROM geo_multipolygon;
|
||||
```
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─mpg─────────────────────────────────────────────────────────────────────────────────────────────┬─toTypeName(mpg)─┐
|
||||
│ [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] │ MultiPolygon │
|
||||
└─────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/data-types/geo/) <!--hide-->
|
@ -21,7 +21,11 @@ The following aggregate functions are supported:
|
||||
- [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md)
|
||||
- [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md)
|
||||
|
||||
Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function.
|
||||
|
||||
!!! note "Note"
|
||||
Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes.
|
||||
|
||||
`SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function.
|
||||
|
||||
**Parameters**
|
||||
|
||||
@ -31,11 +35,7 @@ Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
CREATE TABLE t
|
||||
(
|
||||
column1 SimpleAggregateFunction(sum, UInt64),
|
||||
column2 SimpleAggregateFunction(any, String)
|
||||
) ENGINE = ...
|
||||
CREATE TABLE simple (id UInt64, val SimpleAggregateFunction(sum, Double)) ENGINE=AggregatingMergeTree ORDER BY id;
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) <!--hide-->
|
||||
|
@ -771,7 +771,7 @@ This is necessary for searching for pageviews in the corresponding session.
|
||||
|
||||
## formatDateTime {#formatdatetime}
|
||||
|
||||
Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column.
|
||||
Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -830,31 +830,32 @@ Result:
|
||||
└────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) <!--hide-->
|
||||
|
||||
## FROM\_UNIXTIME {#fromunixfime}
|
||||
|
||||
When there is only single argument of integer type, it act in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md).
|
||||
type.
|
||||
Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type.
|
||||
|
||||
For example:
|
||||
**Example:**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT FROM_UNIXTIME(423543535)
|
||||
SELECT FROM_UNIXTIME(423543535);
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─FROM_UNIXTIME(423543535)─┐
|
||||
│ 1983-06-04 10:58:55 │
|
||||
└──────────────────────────┘
|
||||
```
|
||||
|
||||
When there are two arguments, first is integer or DateTime, second is constant format string, it act in the same way as `formatDateTime` and return `String` type.
|
||||
When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type.
|
||||
|
||||
For example:
|
||||
|
||||
```sql
|
||||
SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime
|
||||
SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime;
|
||||
```
|
||||
|
||||
```text
|
||||
@ -1006,3 +1007,5 @@ Result:
|
||||
│ 2020-01-01 │
|
||||
└────────────────────────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) <!--hide-->
|
||||
|
@ -55,7 +55,7 @@ CREATE TABLE encryption_test
|
||||
`comment` String,
|
||||
`secret` String
|
||||
)
|
||||
ENGINE = Memory
|
||||
ENGINE = Memory;
|
||||
```
|
||||
|
||||
Insert some data (please avoid storing the keys/ivs in the database as this undermines the whole concept of encryption), also storing 'hints' is unsafe too and used only for illustrative purposes:
|
||||
@ -110,7 +110,7 @@ Result:
|
||||
|
||||
Compatible with mysql encryption and resulting ciphertext can be decrypted with [AES_DECRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-decrypt) function.
|
||||
|
||||
Will produce same ciphertext as `encrypt` on equal inputs. But when `key` or `iv` are longer than they should normally be, `aes_encrypt_mysql` will stick to what MySQL's `aes_encrypt` does: 'fold' `key` and ignore excess bits of `IV`.
|
||||
Will produce the same ciphertext as `encrypt` on equal inputs. But when `key` or `iv` are longer than they should normally be, `aes_encrypt_mysql` will stick to what MySQL's `aes_encrypt` does: 'fold' `key` and ignore excess bits of `iv`.
|
||||
|
||||
Supported encryption modes:
|
||||
|
||||
@ -138,7 +138,6 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv])
|
||||
|
||||
- Ciphertext binary string. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
|
||||
**Examples**
|
||||
|
||||
Given equal input `encrypt` and `aes_encrypt_mysql` produce the same ciphertext:
|
||||
@ -157,7 +156,6 @@ Result:
|
||||
└───────────────────┘
|
||||
```
|
||||
|
||||
|
||||
But `encrypt` fails when `key` or `iv` is longer than expected:
|
||||
|
||||
Query:
|
||||
@ -252,7 +250,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad])
|
||||
|
||||
**Examples**
|
||||
|
||||
Re-using table from [encrypt](./encryption-functions.md#encrypt).
|
||||
Re-using table from [encrypt](#encrypt).
|
||||
|
||||
Query:
|
||||
|
||||
@ -284,6 +282,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─comment─────────────────────────────┬─plaintext─┐
|
||||
│ aes-256-cfb128 no IV │ Secret │
|
||||
@ -294,7 +293,7 @@ Result:
|
||||
└─────────────────────────────────────┴───────────┘
|
||||
```
|
||||
|
||||
Notice how only portion of the data was properly decrypted, and the rest is gibberish since either `mode`, `key`, or `iv` were different upon encryption.
|
||||
Notice how only a portion of the data was properly decrypted, and the rest is gibberish since either `mode`, `key`, or `iv` were different upon encryption.
|
||||
|
||||
## aes_decrypt_mysql {#aes_decrypt_mysql}
|
||||
|
||||
@ -331,6 +330,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv])
|
||||
**Examples**
|
||||
|
||||
Let's decrypt data we've previously encrypted with MySQL:
|
||||
|
||||
``` sql
|
||||
mysql> SET block_encryption_mode='aes-256-cfb128';
|
||||
Query OK, 0 rows affected (0.00 sec)
|
||||
@ -345,11 +345,13 @@ mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviv
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─plaintext─┐
|
||||
│ Secret │
|
||||
|
@ -9,7 +9,7 @@ Hash functions can be used for the deterministic pseudo-random shuffling of elem
|
||||
|
||||
## halfMD5 {#hash-functions-halfmd5}
|
||||
|
||||
[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
|
||||
[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
|
||||
|
||||
``` sql
|
||||
halfMD5(par1, ...)
|
||||
@ -54,7 +54,7 @@ sipHash64(par1,...)
|
||||
|
||||
This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function.
|
||||
|
||||
Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
|
||||
Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
|
||||
|
||||
1. After hashing all the input parameters, the function gets the array of hashes.
|
||||
2. Function takes the first and the second elements and calculates a hash for the array of them.
|
||||
|
@ -174,4 +174,129 @@ Result:
|
||||
└──────────────────────────────┴───────────────────────────────────┘
|
||||
```
|
||||
|
||||
## mapContains {#mapcontains}
|
||||
|
||||
Determines whether the `map` contains the `key` parameter.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
mapContains(map, key)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `map` — Map. [Map](../../sql-reference/data-types/map.md).
|
||||
- `key` — Key. Type matches the type of keys of `map` parameter.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- `1` if `map` contains `key`, `0` if not.
|
||||
|
||||
Type: [UInt8](../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapContains(a, 'name') FROM test;
|
||||
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─mapContains(a, 'name')─┐
|
||||
│ 1 │
|
||||
│ 0 │
|
||||
└────────────────────────┘
|
||||
```
|
||||
|
||||
## mapKeys {#mapkeys}
|
||||
|
||||
Returns all keys from the `map` parameter.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
mapKeys(map)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `map` — Map. [Map](../../sql-reference/data-types/map.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Array containing all keys from the `map`.
|
||||
|
||||
Type: [Array](../../sql-reference/data-types/array.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapKeys(a) FROM test;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─mapKeys(a)────────────┐
|
||||
│ ['name','age'] │
|
||||
│ ['number','position'] │
|
||||
└───────────────────────┘
|
||||
```
|
||||
|
||||
## mapValues {#mapvalues}
|
||||
|
||||
Returns all values from the `map` parameter.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
mapKeys(map)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `map` — Map. [Map](../../sql-reference/data-types/map.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Array containing all the values from `map`.
|
||||
|
||||
Type: [Array](../../sql-reference/data-types/array.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapValues(a) FROM test;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─mapValues(a)─────┐
|
||||
│ ['eleven','11'] │
|
||||
│ ['twelve','6.0'] │
|
||||
└──────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/sql-reference/functions/tuple-map-functions/) <!--hide-->
|
||||
|
@ -81,5 +81,5 @@ The `TTL` is no longer there, so the second row is not deleted:
|
||||
|
||||
### See Also
|
||||
|
||||
- More about the [TTL-expression](../../../sql-reference/statements/create/table#ttl-expression).
|
||||
- Modify column [with TTL](../../../sql-reference/statements/alter/column#alter_modify-column).
|
||||
- More about the [TTL-expression](../../../../sql-reference/statements/create/table#ttl-expression).
|
||||
- Modify column [with TTL](../../../../sql-reference/statements/alter/column#alter_modify-column).
|
||||
|
@ -4,10 +4,8 @@ toc_title: ALL
|
||||
|
||||
# ALL Clause {#select-all}
|
||||
|
||||
`SELECT ALL` is identical to `SELECT` without `DISTINCT`.
|
||||
If there are multiple matching rows in the table, then `ALL` returns all of them. `SELECT ALL` is identical to `SELECT` without `DISTINCT`. If both `ALL` and `DISTINCT` specified, exception will be thrown.
|
||||
|
||||
- If `ALL` specified, ignore it.
|
||||
- If both `ALL` and `DISTINCT` specified, exception will be thrown.
|
||||
|
||||
`ALL` can also be specified inside aggregate function with the same effect(noop), for instance:
|
||||
|
||||
@ -19,3 +17,5 @@ equals to
|
||||
```sql
|
||||
SELECT sum(number) FROM numbers(10);
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/all) <!--hide-->
|
||||
|
@ -31,21 +31,26 @@ SETTINGS
|
||||
[kafka_schema = '',]
|
||||
[kafka_num_consumers = N,]
|
||||
[kafka_skip_broken_messages = N]
|
||||
[kafka_commit_every_batch = 0,]
|
||||
[kafka_thread_per_consumer = 0]
|
||||
```
|
||||
|
||||
Обязательные параметры:
|
||||
|
||||
- `kafka_broker_list` – перечень брокеров, разделенный запятыми (`localhost:9092`).
|
||||
- `kafka_topic_list` – перечень необходимых топиков Kafka.
|
||||
- `kafka_group_name` – группа потребителя Kafka. Отступы для чтения отслеживаются для каждой группы отдельно. Если необходимо, чтобы сообщения не повторялись на кластере, используйте везде одно имя группы.
|
||||
- `kafka_format` – формат сообщений. Названия форматов должны быть теми же, что можно использовать в секции `FORMAT`, например, `JSONEachRow`. Подробнее читайте в разделе [Форматы](../../../interfaces/formats.md).
|
||||
- `kafka_broker_list` — перечень брокеров, разделенный запятыми (`localhost:9092`).
|
||||
- `kafka_topic_list` — перечень необходимых топиков Kafka.
|
||||
- `kafka_group_name` — группа потребителя Kafka. Отступы для чтения отслеживаются для каждой группы отдельно. Если необходимо, чтобы сообщения не повторялись на кластере, используйте везде одно имя группы.
|
||||
- `kafka_format` — формат сообщений. Названия форматов должны быть теми же, что можно использовать в секции `FORMAT`, например, `JSONEachRow`. Подробнее читайте в разделе [Форматы](../../../interfaces/formats.md).
|
||||
|
||||
Опциональные параметры:
|
||||
|
||||
- `kafka_row_delimiter` – символ-разделитель записей (строк), которым завершается сообщение.
|
||||
- `kafka_schema` – опциональный параметр, необходимый, если используется формат, требующий определения схемы. Например, [Cap’n Proto](https://capnproto.org/) требует путь к файлу со схемой и название корневого объекта `schema.capnp:Message`.
|
||||
- `kafka_num_consumers` – количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя.
|
||||
- `kafka_skip_broken_messages` – максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0.
|
||||
- `kafka_row_delimiter` — символ-разделитель записей (строк), которым завершается сообщение.
|
||||
- `kafka_schema` — опциональный параметр, необходимый, если используется формат, требующий определения схемы. Например, [Cap’n Proto](https://capnproto.org/) требует путь к файлу со схемой и название корневого объекта `schema.capnp:Message`.
|
||||
- `kafka_num_consumers` — количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя.
|
||||
- `kafka_max_block_size` — максимальный размер пачек (в сообщениях) для poll (по умолчанию `max_block_size`).
|
||||
- `kafka_skip_broken_messages` — максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0.
|
||||
- `kafka_commit_every_batch` — включает или отключает режим записи каждой принятой и обработанной пачки по отдельности вместо единой записи целого блока (по умолчанию `0`).
|
||||
- `kafka_thread_per_consumer` — включает или отключает предоставление отдельного потока каждому потребителю (по умолчанию `0`). При включенном режиме каждый потребитель сбрасывает данные независимо и параллельно, при отключённом — строки с данными от нескольких потребителей собираются в один блок.
|
||||
|
||||
Примеры
|
||||
|
||||
|
@ -22,6 +22,7 @@ toc_title: "\u041a\u043b\u0438\u0435\u043d\u0442\u0441\u043a\u0438\u0435\u0020\u
|
||||
- [seva-code/php-click-house-client](https://packagist.org/packages/seva-code/php-click-house-client)
|
||||
- [SeasClick C++ client](https://github.com/SeasX/SeasClick)
|
||||
- [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel)
|
||||
- [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php)
|
||||
- Go
|
||||
- [clickhouse](https://github.com/kshvakov/clickhouse/)
|
||||
- [go-clickhouse](https://github.com/roistat/go-clickhouse)
|
||||
|
@ -1937,6 +1937,21 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1;
|
||||
|
||||
Значение по умолчанию: 16.
|
||||
|
||||
## background_message_broker_schedule_pool_size {#background_message_broker_schedule_pool_size}
|
||||
|
||||
Задает количество потоков для фонового потокового вывода сообщений. Настройка применяется при запуске сервера ClickHouse и не может быть изменена в пользовательском сеансе.
|
||||
|
||||
Допустимые значения:
|
||||
|
||||
- Положительное целое число.
|
||||
|
||||
Значение по умолчанию: 16.
|
||||
|
||||
**Смотрите также**
|
||||
|
||||
- Движок [Kafka](../../engines/table-engines/integrations/kafka.md#kafka).
|
||||
- Движок [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine).
|
||||
|
||||
## format_avro_schema_registry_url {#format_avro_schema_registry_url}
|
||||
|
||||
Задает URL реестра схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html) для использования с форматом [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent).
|
||||
@ -2537,4 +2552,15 @@ SELECT * FROM test2;
|
||||
|
||||
Обратите внимание на то, что эта настройка влияет на поведение [материализованных представлений](../../sql-reference/statements/create/view.md#materialized) и БД [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md).
|
||||
|
||||
## allow_experimental_geo_types {#allow-experimental-geo-types}
|
||||
|
||||
Разрешает использование экспериментальных типов данных для работы с [географическими структурами](../../sql-reference/data-types/geo.md).
|
||||
|
||||
Возможные значения:
|
||||
|
||||
- 0 — Использование типов данных для работы с географическими структурами не поддерживается.
|
||||
- 1 — Использование типов данных для работы с географическими структурами поддерживается.
|
||||
|
||||
Значение по умолчанию: `0`.
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->
|
||||
|
@ -9,25 +9,54 @@ toc_title: "\u0421\u0438\u0441\u0442\u0435\u043c\u043d\u044b\u0435\u0020\u0442\u
|
||||
|
||||
Системные таблицы содержат информацию о:
|
||||
|
||||
- Состоянии сервера, процессов и окружении.
|
||||
- Внутренних процессах сервера.
|
||||
- состоянии сервера, процессов и окружении.
|
||||
- внутренних процессах сервера.
|
||||
|
||||
Системные таблицы:
|
||||
|
||||
- Находятся в базе данных `system`.
|
||||
- Доступны только для чтения данных.
|
||||
- Не могут быть удалены или изменены, но их можно отсоединить.
|
||||
- находятся в базе данных `system`.
|
||||
- доступны только для чтения данных.
|
||||
- не могут быть удалены или изменены, но их можно отсоединить.
|
||||
|
||||
Системные таблицы `metric_log`, `query_log`, `query_thread_log`, `trace_log` системные таблицы хранят данные в файловой системе. Остальные системные таблицы хранят свои данные в оперативной памяти. Сервер ClickHouse создает такие системные таблицы при запуске.
|
||||
Большинство системных таблиц хранят свои данные в оперативной памяти. Сервер ClickHouse создает эти системные таблицы при старте.
|
||||
|
||||
В отличие от других системных таблиц, таблицы с системными логами [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash-log.md) и [text_log](../../operations/system-tables/text_log.md) используют движок таблиц [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) и по умолчанию хранят свои данные в файловой системе. Если удалить таблицу из файловой системы, сервер ClickHouse снова создаст пустую таблицу во время следующей записи данных. Если схема системной таблицы изменилась в новом релизе, то ClickHouse переименует текущую таблицу и создаст новую.
|
||||
|
||||
Таблицы с системными логами `log` можно настроить, создав конфигурационный файл с тем же именем, что и таблица в разделе `/etc/clickhouse-server/config.d/`, или указав соответствующие элементы в `/etc/clickhouse-server/config.xml`. Настраиваться могут следующие элементы:
|
||||
|
||||
- `database` — база данных, к которой принадлежит системная таблица. Эта опция на текущий момент устарела. Все системные таблицы находятся в базе данных `system`.
|
||||
- `table` — таблица для добавления данных.
|
||||
- `partition_by` — [ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md).
|
||||
- `ttl` — [время жизни](../../sql-reference/statements/alter/ttl.md) таблицы.
|
||||
- `flush_interval_milliseconds` — интервал сброса данных на диск, в миллисекундах.
|
||||
- `engine` — полное имя движка (начиная с `ENGINE =` ) с параметрами. Эта опция противоречит `partition_by` и `ttl`. Если указать оба параметра вместе, сервер вернет ошибку и завершит работу.
|
||||
|
||||
Пример:
|
||||
|
||||
```xml
|
||||
<yandex>
|
||||
<query_log>
|
||||
<database>system</database>
|
||||
<table>query_log</table>
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<ttl>event_date + INTERVAL 30 DAY DELETE</ttl>
|
||||
<!--
|
||||
<engine>ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024</engine>
|
||||
-->
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_log>
|
||||
</yandex>
|
||||
```
|
||||
|
||||
По умолчанию размер таблицы не ограничен. Управлять размером таблицы можно используя [TTL](../../sql-reference/statements/alter/ttl.md#manipuliatsii-s-ttl-tablitsy) для удаления устаревших записей журнала. Также вы можете использовать функцию партиционирования для таблиц `MergeTree`.
|
||||
|
||||
### Источники системных показателей
|
||||
|
||||
Для сбора системных показателей сервер ClickHouse использует:
|
||||
|
||||
- Возможности `CAP_NET_ADMIN`.
|
||||
- возможности `CAP_NET_ADMIN`.
|
||||
- [procfs](https://ru.wikipedia.org/wiki/Procfs) (только Linux).
|
||||
|
||||
**procfs**
|
||||
|
||||
Если для сервера ClickHouse не включено `CAP_NET_ADMIN`, он пытается обратиться к `ProcfsMetricsProvider`. `ProcfsMetricsProvider` позволяет собирать системные показатели для каждого запроса (для CPU и I/O).
|
||||
|
||||
|
@ -21,7 +21,8 @@ toc_title: clickhouse-local
|
||||
Основной формат вызова:
|
||||
|
||||
``` bash
|
||||
$ clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" -q "query"
|
||||
$ clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" \
|
||||
--query "query"
|
||||
```
|
||||
|
||||
Ключи команды:
|
||||
@ -76,7 +77,9 @@ $ clickhouse-local --query "
|
||||
1 2
|
||||
```
|
||||
|
||||
А теперь давайте выведем на экран объём оперативной памяти, занимаемой пользователями (Unix):
|
||||
Объём оперативной памяти, занимаемой процессами, которые запустил пользователь (Unix):
|
||||
|
||||
Запрос:
|
||||
|
||||
``` bash
|
||||
$ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \
|
||||
@ -85,6 +88,8 @@ $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \
|
||||
FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty"
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec.
|
||||
┏━━━━━━━━━━┳━━━━━━━━━━┓
|
||||
|
@ -239,7 +239,7 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `window` — ширина скользящего окна по времени в секундах. [UInt](../../sql-reference/aggregate-functions/parametric-functions.md).
|
||||
- `window` — ширина скользящего окна по времени. Единица измерения зависит от `timestamp` и может варьироваться. Должно соблюдаться условие `timestamp события cond2 <= timestamp события cond1 + window`.
|
||||
- `mode` - необязательный параметр. Если установлено значение `'strict'`, то функция `windowFunnel()` применяет условия только для уникальных значений.
|
||||
- `timestamp` — имя столбца, содержащего временные отметки. [Date](../../sql-reference/aggregate-functions/parametric-functions.md), [DateTime](../../sql-reference/aggregate-functions/parametric-functions.md#data_type-datetime) и другие параметры с типом `Integer`. В случае хранения меток времени в столбцах с типом `UInt64`, максимально допустимое значение соответствует ограничению для типа `Int64`, т.е. равно `2^63-1`.
|
||||
- `cond` — условия или данные, описывающие цепочку событий. [UInt8](../../sql-reference/aggregate-functions/parametric-functions.md).
|
||||
|
106
docs/ru/sql-reference/data-types/geo.md
Normal file
106
docs/ru/sql-reference/data-types/geo.md
Normal file
@ -0,0 +1,106 @@
|
||||
---
|
||||
toc_priority: 62
|
||||
toc_title: Географические структуры
|
||||
---
|
||||
|
||||
# Типы данных для работы с географическими структурами {#geo-data-types}
|
||||
|
||||
ClickHouse поддерживает типы данных для отображения географических объектов — точек (местоположений), территорий и т.п.
|
||||
|
||||
!!! warning "Предупреждение"
|
||||
Сейчас использование типов данных для работы с географическими структурами является экспериментальной возможностью. Чтобы использовать эти типы данных, включите настройку `allow_experimental_geo_types = 1`.
|
||||
|
||||
**См. также**
|
||||
- [Хранение географических структур данных](https://ru.wikipedia.org/wiki/GeoJSON).
|
||||
- Настройка [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types).
|
||||
|
||||
## Point {#point-data-type}
|
||||
|
||||
Тип `Point` (точка) определяется парой координат X и Y и хранится в виде кортежа [Tuple](tuple.md)([Float64](float.md), [Float64](float.md)).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_point (p Point) ENGINE = Memory();
|
||||
INSERT INTO geo_point VALUES((10, 10));
|
||||
SELECT p, toTypeName(p) FROM geo_point;
|
||||
```
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─p─────┬─toTypeName(p)─┐
|
||||
│ (10,10) │ Point │
|
||||
└───────┴───────────────┘
|
||||
```
|
||||
|
||||
## Ring {#ring-data-type}
|
||||
|
||||
Тип `Ring` описывает простой многоугольник без внутренних областей (дыр) и хранится в виде массива точек: [Array](array.md)([Point](#point-data-type)).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_ring (r Ring) ENGINE = Memory();
|
||||
INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
SELECT r, toTypeName(r) FROM geo_ring;
|
||||
```
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─r─────────────────────────────┬─toTypeName(r)─┐
|
||||
│ [(0,0),(10,0),(10,10),(0,10)] │ Ring │
|
||||
└───────────────────────────────┴───────────────┘
|
||||
```
|
||||
|
||||
## Polygon {#polygon-data-type}
|
||||
|
||||
Тип `Polygon` описывает многоугольник с внутренними областями (дырами) и хранится в виде массива: [Array](array.md)([Ring](#ring-data-type)). Первый элемент массива описывает внешний многоугольник (контур), а остальные элементы описывают дыры.
|
||||
|
||||
**Пример**
|
||||
|
||||
Запись в этой таблице описывает многоугольник с одной дырой:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory();
|
||||
INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]);
|
||||
SELECT pg, toTypeName(pg) FROM geo_polygon;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─pg────────────────────────────────────────────────────────────┬─toTypeName(pg)─┐
|
||||
│ [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] │ Polygon │
|
||||
└───────────────────────────────────────────────────────────────┴────────────────┘
|
||||
```
|
||||
|
||||
## MultiPolygon {#multipolygon-data-type}
|
||||
|
||||
Тип `MultiPolygon` описывает элемент, состоящий из нескольких простых многоугольников (полигональную сетку). Он хранится в виде массива многоугольников: [Array](array.md)([Polygon](#polygon-data-type)).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запись в этой таблице описывает элемент, состоящий из двух многоугольников — первый без дыр, а второй с одной дырой:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_geo_types = 1;
|
||||
CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory();
|
||||
INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]);
|
||||
SELECT mpg, toTypeName(mpg) FROM geo_multipolygon;
|
||||
```
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─mpg─────────────────────────────────────────────────────────────────────────────────────────────┬─toTypeName(mpg)─┐
|
||||
│ [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] │ MultiPolygon │
|
||||
└─────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────┘
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/data-types/geo/) <!--hide-->
|
@ -1,8 +1,9 @@
|
||||
# SimpleAggregateFunction {#data-type-simpleaggregatefunction}
|
||||
# SimpleAggregateFunction(func, type) {#data-type-simpleaggregatefunction}
|
||||
|
||||
`SimpleAggregateFunction(name, types_of_arguments…)` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we don’t have to store and process any extra data.
|
||||
Хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, которые обладают следующим свойством: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк,
|
||||
а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому хранить и обрабатывать какие-либо дополнительные данные не требуется.
|
||||
|
||||
The following aggregate functions are supported:
|
||||
Поддерживаются следующие агрегатные функции:
|
||||
|
||||
- [`any`](../../sql-reference/aggregate-functions/reference/any.md#agg_function-any)
|
||||
- [`anyLast`](../../sql-reference/aggregate-functions/reference/anylast.md#anylastx)
|
||||
@ -15,22 +16,24 @@ The following aggregate functions are supported:
|
||||
- [`groupBitXor`](../../sql-reference/aggregate-functions/reference/groupbitxor.md#groupbitxor)
|
||||
- [`groupArrayArray`](../../sql-reference/aggregate-functions/reference/grouparray.md#agg_function-grouparray)
|
||||
- [`groupUniqArrayArray`](../../sql-reference/aggregate-functions/reference/groupuniqarray.md#groupuniqarray)
|
||||
- [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap)
|
||||
- [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap)
|
||||
- [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap)
|
||||
|
||||
Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function.
|
||||
!!! note "Примечание"
|
||||
Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому комбинаторы [-Merge](../../sql-reference/aggregate-functions/combinators.md#aggregate_functions_combinators-merge) и [-State]((../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) не требуются.
|
||||
|
||||
`SimpleAggregateFunction` имеет лучшую производительность, чем `AggregateFunction` с той же агрегатной функцией.
|
||||
|
||||
**Parameters**
|
||||
**Параметры**
|
||||
|
||||
- Name of the aggregate function.
|
||||
- Types of the aggregate function arguments.
|
||||
- `func` — имя агрегатной функции.
|
||||
- `type` — типы аргументов агрегатной функции.
|
||||
|
||||
**Example**
|
||||
**Пример**
|
||||
|
||||
``` sql
|
||||
CREATE TABLE t
|
||||
(
|
||||
column1 SimpleAggregateFunction(sum, UInt64),
|
||||
column2 SimpleAggregateFunction(any, String)
|
||||
) ENGINE = ...
|
||||
CREATE TABLE simple (id UInt64, val SimpleAggregateFunction(sum, Double)) ENGINE=AggregatingMergeTree ORDER BY id;
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) <!--hide-->
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/data-types/simpleaggregatefunction/) <!--hide-->
|
||||
|
@ -572,7 +572,7 @@ SOURCE(CLICKHOUSE(
|
||||
или
|
||||
|
||||
``` sql
|
||||
SOURCE(MONGO(
|
||||
SOURCE(MONGODB(
|
||||
host 'localhost'
|
||||
port 27017
|
||||
user ''
|
||||
|
@ -323,7 +323,9 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d
|
||||
|
||||
Переводит дату-с-временем или дату в число типа UInt16, содержащее номер ISO года. ISO год отличается от обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) ISO год начинается необязательно первого января.
|
||||
|
||||
Пример:
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -331,6 +333,9 @@ SELECT
|
||||
toYear(date),
|
||||
toISOYear(date)
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌───────date─┬─toYear(toDate('2017-01-01'))─┬─toISOYear(toDate('2017-01-01'))─┐
|
||||
│ 2017-01-01 │ 2017 │ 2016 │
|
||||
@ -344,12 +349,18 @@ SELECT
|
||||
|
||||
1 Января 2017 г. - воскресение, т.е. первая ISO неделя 2017 года началась в понедельник 2 января, поэтому 1 января 2017 это последняя неделя 2016 года.
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
toISOWeek(toDate('2017-01-01')) AS ISOWeek20170101,
|
||||
toISOWeek(toDate('2017-01-02')) AS ISOWeek20170102
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─ISOWeek20170101─┬─ISOWeek20170102─┐
|
||||
│ 52 │ 1 │
|
||||
@ -386,10 +397,14 @@ SELECT
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS week1, toWeek(date,9) AS week9;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌───────date─┬─week0─┬─week1─┬─week9─┐
|
||||
│ 2016-12-27 │ 52 │ 52 │ 1 │
|
||||
@ -405,10 +420,14 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐
|
||||
│ 2016-12-27 │ 201652 │ 201652 │ 201701 │
|
||||
@ -591,7 +610,7 @@ dateDiff('unit', startdate, enddate, [timezone])
|
||||
SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
|
||||
```
|
||||
|
||||
Ответ:
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐
|
||||
@ -672,10 +691,10 @@ formatDateTime(Time, Format\[, Timezone\])
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT formatDateTime(toDate('2010-01-04'), '%g')
|
||||
SELECT formatDateTime(toDate('2010-01-04'), '%g');
|
||||
```
|
||||
|
||||
Ответ:
|
||||
Результат:
|
||||
|
||||
```
|
||||
┌─formatDateTime(toDate('2010-01-04'), '%g')─┐
|
||||
@ -683,4 +702,43 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g')
|
||||
└────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## FROM\_UNIXTIME {#fromunixtime}
|
||||
|
||||
Функция преобразует Unix timestamp в календарную дату и время.
|
||||
|
||||
**Примеры**
|
||||
|
||||
Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md).
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT FROM_UNIXTIME(423543535);
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─FROM_UNIXTIME(423543535)─┐
|
||||
│ 1983-06-04 10:58:55 │
|
||||
└──────────────────────────┘
|
||||
```
|
||||
|
||||
В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает также, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─DateTime────────────┐
|
||||
│ 2009-02-11 14:42:23 │
|
||||
└─────────────────────┘
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/date_time_functions/) <!--hide-->
|
||||
|
@ -11,7 +11,7 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438 \u0434\u043b\u044f \u0448
|
||||
|
||||
Длина инициализирующего вектора всегда 16 байт (лишнии байты игнорируются).
|
||||
|
||||
Обратите внимание, что эти функции работают медленно.
|
||||
Обратите внимание, что до версии Clickhouse 21.1 эти функции работали медленно.
|
||||
|
||||
## encrypt {#encrypt}
|
||||
|
||||
@ -41,7 +41,7 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad])
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Зашифрованная строка. [String](../../sql-reference/data-types/string.md#string).
|
||||
- Бинарная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
**Примеры**
|
||||
|
||||
@ -52,57 +52,38 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad])
|
||||
``` sql
|
||||
CREATE TABLE encryption_test
|
||||
(
|
||||
input String,
|
||||
key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'),
|
||||
iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'),
|
||||
key32 String DEFAULT substring(key, 1, 32),
|
||||
key24 String DEFAULT substring(key, 1, 24),
|
||||
key16 String DEFAULT substring(key, 1, 16)
|
||||
) Engine = Memory;
|
||||
`comment` String,
|
||||
`secret` String
|
||||
)
|
||||
ENGINE = Memory;
|
||||
```
|
||||
|
||||
Вставим эти данные:
|
||||
Вставим некоторые данные (замечание: не храните ключи или инициализирующие векторы в базе данных, так как это компрометирует всю концепцию шифрования), также хранение "подсказок" небезопасно и используется только для наглядности:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?');
|
||||
INSERT INTO encryption_test VALUES('aes-256-cfb128 no IV', encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212')),\
|
||||
('aes-256-cfb128 no IV, different key', encrypt('aes-256-cfb128', 'Secret', 'keykeykeykeykeykeykeykeykeykeyke')),\
|
||||
('aes-256-cfb128 with IV', encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv')),\
|
||||
('aes-256-cbc no IV', encrypt('aes-256-cbc', 'Secret', '12345678910121314151617181920212'));
|
||||
```
|
||||
|
||||
Пример без `iv`:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-128-ecb' AS mode, hex(encrypt(mode, input, key16)) FROM encryption_test;
|
||||
SELECT comment, hex(secret) FROM encryption_test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─hex(encrypt('aes-128-ecb', input, key16))────────────────────────┐
|
||||
│ aes-128-ecb │ 4603E6862B0D94BBEC68E0B0DF51D60F │
|
||||
│ aes-128-ecb │ 3004851B86D3F3950672DE7085D27C03 │
|
||||
│ aes-128-ecb │ E807F8C8D40A11F65076361AFC7D8B68D8658C5FAA6457985CAA380F16B3F7E4 │
|
||||
└─────────────┴──────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
Пример с `iv`:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-256-ctr' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encryption_test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─hex(encrypt('aes-256-ctr', input, key32, iv))─┐
|
||||
│ aes-256-ctr │ │
|
||||
│ aes-256-ctr │ 7FB039F7 │
|
||||
│ aes-256-ctr │ 5CBD20F7ABD3AC41FCAA1A5C0E119E2B325949 │
|
||||
└─────────────┴───────────────────────────────────────────────┘
|
||||
┌─comment─────────────────────────────┬─hex(secret)──────────────────────┐
|
||||
│ aes-256-cfb128 no IV │ B4972BDC4459 │
|
||||
│ aes-256-cfb128 no IV, different key │ 2FF57C092DC9 │
|
||||
│ aes-256-cfb128 with IV │ 5E6CB398F653 │
|
||||
│ aes-256-cbc no IV │ 1BC0629A92450D9E73A00E7D02CF4142 │
|
||||
└─────────────────────────────────────┴──────────────────────────────────┘
|
||||
```
|
||||
|
||||
Пример в режиме `-gcm`:
|
||||
@ -110,41 +91,27 @@ SELECT 'aes-256-ctr' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encrypti
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-256-gcm' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encryption_test;
|
||||
INSERT INTO encryption_test VALUES('aes-256-gcm', encrypt('aes-256-gcm', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv')), \
|
||||
('aes-256-gcm with AAD', encrypt('aes-256-gcm', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv', 'aad'));
|
||||
|
||||
SELECT comment, hex(secret) FROM encryption_test WHERE comment LIKE '%gcm%';
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─hex(encrypt('aes-256-gcm', input, key32, iv))──────────────────────────┐
|
||||
│ aes-256-gcm │ E99DBEBC01F021758352D7FBD9039EFA │
|
||||
│ aes-256-gcm │ 8742CE3A7B0595B281C712600D274CA881F47414 │
|
||||
│ aes-256-gcm │ A44FD73ACEB1A64BDE2D03808A2576EDBB60764CC6982DB9AF2C33C893D91B00C60DC5 │
|
||||
└─────────────┴────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
Пример в режиме `-gcm` и с `aad`:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-192-gcm' AS mode, hex(encrypt(mode, input, key24, iv, 'AAD')) FROM encryption_test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─hex(encrypt('aes-192-gcm', input, key24, iv, 'AAD'))───────────────────┐
|
||||
│ aes-192-gcm │ 04C13E4B1D62481ED22B3644595CB5DB │
|
||||
│ aes-192-gcm │ 9A6CF0FD2B329B04EAD18301818F016DF8F77447 │
|
||||
│ aes-192-gcm │ B961E9FD9B940EBAD7ADDA75C9F198A40797A5EA1722D542890CC976E21113BBB8A7AA │
|
||||
└─────────────┴────────────────────────────────────────────────────────────────────────┘
|
||||
┌─comment──────────────┬─hex(secret)──────────────────────────────────┐
|
||||
│ aes-256-gcm │ A8A3CCBC6426CFEEB60E4EAE03D3E94204C1B09E0254 │
|
||||
│ aes-256-gcm with AAD │ A8A3CCBC6426D9A1017A0A932322F1852260A4AD6837 │
|
||||
└──────────────────────┴──────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## aes_encrypt_mysql {#aes_encrypt_mysql}
|
||||
|
||||
Совместима с шифрованием myqsl, результат может быть расшифрован функцией [AES_DECRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-decrypt).
|
||||
|
||||
При одинаковых входящих значениях зашифрованный текст будет совпадать с результатом, возвращаемым функцией `encrypt`. Однако если `key` или `iv` длиннее, чем должны быть, `aes_encrypt_mysql` будет работать аналогично функции `aes_encrypt` в MySQL: свернет ключ и проигнорирует лишнюю часть `iv`.
|
||||
|
||||
Функция поддерживает шифрофание данных следующими режимами:
|
||||
|
||||
- aes-128-ecb, aes-192-ecb, aes-256-ecb
|
||||
@ -156,7 +123,7 @@ SELECT 'aes-192-gcm' AS mode, hex(encrypt(mode, input, key24, iv, 'AAD')) FROM e
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
``` sql
|
||||
aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv])
|
||||
```
|
||||
|
||||
@ -164,78 +131,96 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv])
|
||||
|
||||
- `mode` — режим шифрования. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `plaintext` — текст, который будет зашифрован. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `key` — ключ шифрования. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `iv` — инициализирующий вектор. Необязателен. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `key` — ключ шифрования. Если ключ длиннее, чем требует режим шифрования, производится специфичная для MySQL свертка ключа. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `iv` — инициализирующий вектор. Необязателен, учитываются только первые 16 байтов. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Зашифрованная строка. [String](../../sql-reference/data-types/string.md#string).
|
||||
- Бинарная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
**Примеры**
|
||||
|
||||
Создадим такую таблицу:
|
||||
При одинаковых входящих значениях результаты шифрования у функций `encrypt` и `aes_encrypt_mysql` совпадают.
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE encryption_test
|
||||
(
|
||||
input String,
|
||||
key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'),
|
||||
iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'),
|
||||
key32 String DEFAULT substring(key, 1, 32),
|
||||
key24 String DEFAULT substring(key, 1, 24),
|
||||
key16 String DEFAULT substring(key, 1, 16)
|
||||
) Engine = Memory;
|
||||
```
|
||||
|
||||
Вставим эти данные:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?');
|
||||
```
|
||||
|
||||
Пример без `iv`:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-128-cbc' AS mode, hex(aes_encrypt_mysql(mode, input, key32)) FROM encryption_test;
|
||||
SELECT encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv') = aes_encrypt_mysql('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv') AS ciphertexts_equal;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─hex(aes_encrypt_mysql('aes-128-cbc', input, key32))──────────────┐
|
||||
│ aes-128-cbc │ FEA8CFDE6EE2C6E7A2CC6ADDC9F62C83 │
|
||||
│ aes-128-cbc │ 78B16CD4BE107660156124C5FEE6454A │
|
||||
│ aes-128-cbc │ 67C0B119D96F18E2823968D42871B3D179221B1E7EE642D628341C2B29BA2E18 │
|
||||
└─────────────┴──────────────────────────────────────────────────────────────────┘
|
||||
┌─ciphertexts_equal─┐
|
||||
│ 1 │
|
||||
└───────────────────┘
|
||||
```
|
||||
|
||||
Пример с `iv`:
|
||||
Функция `encrypt` генерирует исключение, если `key` или `iv` длиннее чем нужно:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-256-cfb128' AS mode, hex(aes_encrypt_mysql(mode, input, key32, iv)) FROM encryption_test;
|
||||
SELECT encrypt('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123');
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode───────────┬─hex(aes_encrypt_mysql('aes-256-cfb128', input, key32, iv))─┐
|
||||
│ aes-256-cfb128 │ │
|
||||
│ aes-256-cfb128 │ 7FB039F7 │
|
||||
│ aes-256-cfb128 │ 5CBD20F7ABD3AC41FCAA1A5C0E119E2BB5174F │
|
||||
└────────────────┴────────────────────────────────────────────────────────────┘
|
||||
Received exception from server (version 21.1.2):
|
||||
Code: 36. DB::Exception: Received from localhost:9000. DB::Exception: Invalid key size: 33 expected 32: While processing encrypt('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123').
|
||||
```
|
||||
|
||||
Однако функция `aes_encrypt_mysql` в аналогичном случае возвращает результат, который может быть обработан MySQL:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT hex(aes_encrypt_mysql('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123')) AS ciphertext;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─ciphertext───┐
|
||||
│ 24E9E4966469 │
|
||||
└──────────────┘
|
||||
```
|
||||
|
||||
Если передать `iv` еще длиннее, результат останется таким же:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT hex(aes_encrypt_mysql('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456')) AS ciphertext
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─ciphertext───┐
|
||||
│ 24E9E4966469 │
|
||||
└──────────────┘
|
||||
```
|
||||
|
||||
Это совпадает с результатом, возвращаемым MySQL при таких же входящих значениях:
|
||||
|
||||
``` sql
|
||||
mysql> SET block_encryption_mode='aes-256-cfb128';
|
||||
Query OK, 0 rows affected (0.00 sec)
|
||||
|
||||
mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456') as ciphertext;
|
||||
+------------------------+
|
||||
| ciphertext |
|
||||
+------------------------+
|
||||
| 0x24E9E4966469 |
|
||||
+------------------------+
|
||||
1 row in set (0.00 sec)
|
||||
```
|
||||
|
||||
## decrypt {#decrypt}
|
||||
|
||||
Функция поддерживает расшифровку данных следующими режимами:
|
||||
Функция расшифровывает зашифрованный текст и может работать в следующих режимах:
|
||||
|
||||
- aes-128-ecb, aes-192-ecb, aes-256-ecb
|
||||
- aes-128-cbc, aes-192-cbc, aes-256-cbc
|
||||
@ -247,7 +232,7 @@ SELECT 'aes-256-cfb128' AS mode, hex(aes_encrypt_mysql(mode, input, key32, iv))
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
``` sql
|
||||
decrypt('mode', 'ciphertext', 'key' [, iv, aad])
|
||||
```
|
||||
|
||||
@ -265,52 +250,58 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad])
|
||||
|
||||
**Примеры**
|
||||
|
||||
Создадим такую таблицу:
|
||||
Рассмотрим таблицу из примера для функции [encrypt](#encrypt).
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE encryption_test
|
||||
(
|
||||
input String,
|
||||
key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'),
|
||||
iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'),
|
||||
key32 String DEFAULT substring(key, 1, 32),
|
||||
key24 String DEFAULT substring(key, 1, 24),
|
||||
key16 String DEFAULT substring(key, 1, 16)
|
||||
) Engine = Memory;
|
||||
```
|
||||
|
||||
Вставим эти данные:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?');
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
|
||||
SELECT 'aes-128-ecb' AS mode, decrypt(mode, encrypt(mode, input, key16), key16) FROM encryption_test;
|
||||
SELECT comment, hex(secret) FROM encryption_test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─mode────────┬─decrypt('aes-128-ecb', encrypt('aes-128-ecb', input, key16), key16)─┐
|
||||
│ aes-128-ecb │ │
|
||||
│ aes-128-ecb │ text │
|
||||
│ aes-128-ecb │ What Is ClickHouse? │
|
||||
└─────────────┴─────────────────────────────────────────────────────────────────────┘
|
||||
``` text
|
||||
┌─comment──────────────┬─hex(secret)──────────────────────────────────┐
|
||||
│ aes-256-gcm │ A8A3CCBC6426CFEEB60E4EAE03D3E94204C1B09E0254 │
|
||||
│ aes-256-gcm with AAD │ A8A3CCBC6426D9A1017A0A932322F1852260A4AD6837 │
|
||||
└──────────────────────┴──────────────────────────────────────────────┘
|
||||
┌─comment─────────────────────────────┬─hex(secret)──────────────────────┐
|
||||
│ aes-256-cfb128 no IV │ B4972BDC4459 │
|
||||
│ aes-256-cfb128 no IV, different key │ 2FF57C092DC9 │
|
||||
│ aes-256-cfb128 with IV │ 5E6CB398F653 │
|
||||
│ aes-256-cbc no IV │ 1BC0629A92450D9E73A00E7D02CF4142 │
|
||||
└─────────────────────────────────────┴──────────────────────────────────┘
|
||||
```
|
||||
|
||||
Теперь попытаемся расшифровать эти данные:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920212') as plaintext FROM encryption_test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─comment─────────────────────────────┬─plaintext─┐
|
||||
│ aes-256-cfb128 no IV │ Secret │
|
||||
│ aes-256-cfb128 no IV, different key │ <20>4<EFBFBD>
|
||||
<20> │
|
||||
│ aes-256-cfb128 with IV │ <20><><EFBFBD>6<EFBFBD>~ │
|
||||
│aes-256-cbc no IV │ <20>2*4<>h3c<33>4w<34><77>@
|
||||
└─────────────────────────────────────┴───────────┘
|
||||
```
|
||||
|
||||
Обратите внимание, что только часть данных была расшифрована верно. Оставшаяся часть расшифрована некорректно, так как при шифровании использовались другие значения `mode`, `key`, или `iv`.
|
||||
|
||||
## aes_decrypt_mysql {#aes_decrypt_mysql}
|
||||
|
||||
Совместима с шифрованием myqsl и может расшифровать данные, зашифрованные функцией [AES_ENCRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-encrypt).
|
||||
|
||||
Функция поддерживает расшифровку данных следующими режимами:
|
||||
При одинаковых входящих значениях расшифрованный текст будет совпадать с результатом, возвращаемым функцией `decrypt`. Однако если `key` или `iv` длиннее, чем должны быть, `aes_decrypt_mysql` будет работать аналогично функции `aes_decrypt` в MySQL: свернет ключ и проигнорирует лишнюю часть `iv`.
|
||||
|
||||
Функция поддерживает расшифровку данных в следующих режимах:
|
||||
|
||||
- aes-128-ecb, aes-192-ecb, aes-256-ecb
|
||||
- aes-128-cbc, aes-192-cbc, aes-256-cbc
|
||||
@ -321,7 +312,7 @@ SELECT 'aes-128-ecb' AS mode, decrypt(mode, encrypt(mode, input, key16), key16)
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
``` sql
|
||||
aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv])
|
||||
```
|
||||
|
||||
@ -332,51 +323,39 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv])
|
||||
- `key` — ключ шифрования. [String](../../sql-reference/data-types/string.md#string).
|
||||
- `iv` — инициализирующий вектор. Необязателен. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Расшифрованная строка. [String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
**Примеры**
|
||||
|
||||
Создадим такую таблицу:
|
||||
Расшифруем данные, которые до этого были зашифрованы в MySQL:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE encryption_test
|
||||
(
|
||||
input String,
|
||||
key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'),
|
||||
iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'),
|
||||
key32 String DEFAULT substring(key, 1, 32),
|
||||
key24 String DEFAULT substring(key, 1, 24),
|
||||
key16 String DEFAULT substring(key, 1, 16)
|
||||
) Engine = Memory;
|
||||
```
|
||||
mysql> SET block_encryption_mode='aes-256-cfb128';
|
||||
Query OK, 0 rows affected (0.00 sec)
|
||||
|
||||
Вставим эти данные:
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?');
|
||||
mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456') as ciphertext;
|
||||
+------------------------+
|
||||
| ciphertext |
|
||||
+------------------------+
|
||||
| 0x24E9E4966469 |
|
||||
+------------------------+
|
||||
1 row in set (0.00 sec)
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT 'aes-128-cbc' AS mode, aes_decrypt_mysql(mode, aes_encrypt_mysql(mode, input, key), key) FROM encryption_test;
|
||||
SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─mode────────┬─aes_decrypt_mysql('aes-128-cbc', aes_encrypt_mysql('aes-128-cbc', input, key), key)─┐
|
||||
│ aes-128-cbc │ │
|
||||
│ aes-128-cbc │ text │
|
||||
│ aes-128-cbc │ What Is ClickHouse? │
|
||||
└─────────────┴─────────────────────────────────────────────────────────────────────────────────────┘
|
||||
┌─plaintext─┐
|
||||
│ Secret │
|
||||
└───────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/ru/sql-reference/functions/encryption_functions/) <!--hide-->
|
||||
|
@ -176,4 +176,129 @@ select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type
|
||||
└──────────────────────────────┴───────────────────────────────────┘
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) <!--hide-->
|
||||
## mapContains {#mapcontains}
|
||||
|
||||
Определяет, содержит ли контейнер `map` ключ `key`.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
``` sql
|
||||
mapContains(map, key)
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md).
|
||||
- `key` — ключ. Тип соответстует типу ключей параметра `map`.
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- `1` если `map` включает `key`, иначе `0`.
|
||||
|
||||
Тип: [UInt8](../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapContains(a, 'name') FROM test;
|
||||
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─mapContains(a, 'name')─┐
|
||||
│ 1 │
|
||||
│ 0 │
|
||||
└────────────────────────┘
|
||||
```
|
||||
|
||||
## mapKeys {#mapkeys}
|
||||
|
||||
Возвращает все ключи контейнера `map`.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
mapKeys(map)
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Массив со всеми ключами контейнера `map`.
|
||||
|
||||
Тип: [Array](../../sql-reference/data-types/array.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapKeys(a) FROM test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─mapKeys(a)────────────┐
|
||||
│ ['name','age'] │
|
||||
│ ['number','position'] │
|
||||
└───────────────────────┘
|
||||
```
|
||||
|
||||
## mapValues {#mapvalues}
|
||||
|
||||
Возвращает все значения контейнера `map`.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
mapKeys(map)
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Массив со всеми значениями контейнера `map`.
|
||||
|
||||
Тип: [Array](../../sql-reference/data-types/array.md).
|
||||
|
||||
**Примеры**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'});
|
||||
|
||||
SELECT mapValues(a) FROM test;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─mapValues(a)─────┐
|
||||
│ ['eleven','11'] │
|
||||
│ ['twelve','6.0'] │
|
||||
└──────────────────┘
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/functions/tuple-map-functions/) <!--hide-->
|
||||
|
22
docs/ru/sql-reference/statements/select/all.md
Normal file
22
docs/ru/sql-reference/statements/select/all.md
Normal file
@ -0,0 +1,22 @@
|
||||
---
|
||||
toc_title: ALL
|
||||
---
|
||||
|
||||
# Секция ALL {#select-all}
|
||||
|
||||
Если в таблице несколько совпадающих строк, то `ALL` возвращает все из них. Поведение запроса `SELECT ALL` точно такое же, как и `SELECT` без аргумента `DISTINCT`. Если указаны оба аргумента: `ALL` и `DISTINCT`, функция вернет исключение.
|
||||
|
||||
|
||||
`ALL` может быть указан внутри агрегатной функции, например, результат выполнения запроса:
|
||||
|
||||
```sql
|
||||
SELECT sum(ALL number) FROM numbers(10);
|
||||
```
|
||||
|
||||
равен результату выполнения запроса:
|
||||
|
||||
```sql
|
||||
SELECT sum(number) FROM numbers(10);
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/all) <!--hide-->
|
@ -82,6 +82,7 @@
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
# include <sys/mman.h>
|
||||
# include <sys/ptrace.h>
|
||||
# include <Common/hasLinuxCapability.h>
|
||||
#endif
|
||||
|
||||
@ -480,16 +481,26 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA,
|
||||
"Calculated checksum of the ClickHouse binary ({0}) does not correspond"
|
||||
" to the reference checksum stored in the binary ({1})."
|
||||
" It may indicate one of the following:"
|
||||
" - the file {2} was changed just after startup;"
|
||||
" - the file {2} is damaged on disk due to faulty hardware;"
|
||||
" - the loaded executable is damaged in memory due to faulty hardware;"
|
||||
" - the file {2} was intentionally modified;"
|
||||
" - logical error in code."
|
||||
, calculated_binary_hash, stored_binary_hash, executable_path);
|
||||
/// If program is run under debugger, ptrace will fail.
|
||||
if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1)
|
||||
{
|
||||
/// Program is run under debugger. Modification of it's binary image is ok for breakpoints.
|
||||
LOG_WARNING(log, "Server is run under debugger and its binary image is modified (most likely with breakpoints).",
|
||||
calculated_binary_hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA,
|
||||
"Calculated checksum of the ClickHouse binary ({0}) does not correspond"
|
||||
" to the reference checksum stored in the binary ({1})."
|
||||
" It may indicate one of the following:"
|
||||
" - the file {2} was changed just after startup;"
|
||||
" - the file {2} is damaged on disk due to faulty hardware;"
|
||||
" - the loaded executable is damaged in memory due to faulty hardware;"
|
||||
" - the file {2} was intentionally modified;"
|
||||
" - logical error in code."
|
||||
, calculated_binary_hash, stored_binary_hash, executable_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -118,6 +118,8 @@ public:
|
||||
WhichDataType value_type_to_check(value_type);
|
||||
|
||||
/// Do not promote decimal because of implementation issues of this function design
|
||||
/// Currently we cannot get result column type in case of decimal we cannot get decimal scale
|
||||
/// in method void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
|
||||
/// If we decide to make this function more efficient we should promote decimal type during summ
|
||||
if (value_type_to_check.isDecimal())
|
||||
result_type = value_type_without_nullable;
|
||||
|
@ -101,8 +101,8 @@ endif()
|
||||
list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD})
|
||||
list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON})
|
||||
|
||||
list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp)
|
||||
list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h)
|
||||
list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp)
|
||||
list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h)
|
||||
|
||||
list (APPEND dbms_sources
|
||||
AggregateFunctions/AggregateFunctionFactory.cpp
|
||||
|
@ -538,12 +538,14 @@
|
||||
M(569, MULTIPLE_COLUMNS_SERIALIZED_TO_SAME_PROTOBUF_FIELD) \
|
||||
M(570, DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD) \
|
||||
M(571, DATABASE_REPLICATION_FAILED) \
|
||||
M(572, EPOLL_ERROR) \
|
||||
M(572, TOO_MANY_QUERY_PLAN_OPTIMIZATIONS) \
|
||||
M(573, EPOLL_ERROR) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
M(1001, STD_EXCEPTION) \
|
||||
M(1002, UNKNOWN_EXCEPTION) \
|
||||
M(1003, INVALID_SHARD_ID)
|
||||
M(1003, INVALID_SHARD_ID) \
|
||||
|
||||
/* See END */
|
||||
|
||||
|
@ -391,6 +391,9 @@ public:
|
||||
virtual void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback) = 0;
|
||||
|
||||
/// Expire session and finish all pending requests
|
||||
virtual void finalize() = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -30,7 +30,7 @@ using TestKeeperRequestPtr = std::shared_ptr<TestKeeperRequest>;
|
||||
*
|
||||
* NOTE: You can add various failure modes for better testing.
|
||||
*/
|
||||
class TestKeeper : public IKeeper
|
||||
class TestKeeper final : public IKeeper
|
||||
{
|
||||
public:
|
||||
TestKeeper(const String & root_path_, Poco::Timespan operation_timeout_);
|
||||
@ -83,6 +83,7 @@ public:
|
||||
const Requests & requests,
|
||||
MultiCallback callback) override;
|
||||
|
||||
void finalize() override;
|
||||
|
||||
struct Node
|
||||
{
|
||||
@ -130,7 +131,6 @@ private:
|
||||
|
||||
void pushRequest(RequestInfo && request);
|
||||
|
||||
void finalize();
|
||||
|
||||
ThreadFromGlobalPool processing_thread;
|
||||
|
||||
|
@ -44,7 +44,7 @@ static void check(Coordination::Error code, const std::string & path)
|
||||
}
|
||||
|
||||
|
||||
void ZooKeeper::init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_,
|
||||
void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_,
|
||||
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_)
|
||||
{
|
||||
log = &Poco::Logger::get("ZooKeeper");
|
||||
@ -60,13 +60,16 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
|
||||
if (hosts.empty())
|
||||
throw KeeperException("No hosts passed to ZooKeeper constructor.", Coordination::Error::ZBADARGUMENTS);
|
||||
|
||||
std::vector<std::string> hosts_strings;
|
||||
splitInto<','>(hosts_strings, hosts);
|
||||
Coordination::ZooKeeper::Nodes nodes;
|
||||
nodes.reserve(hosts_strings.size());
|
||||
nodes.reserve(hosts.size());
|
||||
|
||||
Strings shuffled_hosts = hosts;
|
||||
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
|
||||
pcg64 generator(randomSeed());
|
||||
std::shuffle(shuffled_hosts.begin(), shuffled_hosts.end(), generator);
|
||||
|
||||
bool dns_error = false;
|
||||
for (auto & host_string : hosts_strings)
|
||||
for (auto & host_string : shuffled_hosts)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -109,9 +112,9 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
|
||||
Poco::Timespan(0, operation_timeout_ms_ * 1000));
|
||||
|
||||
if (chroot.empty())
|
||||
LOG_TRACE(log, "Initialized, hosts: {}", hosts);
|
||||
LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(hosts, ","));
|
||||
else
|
||||
LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", hosts, chroot);
|
||||
LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(hosts, ","), chroot);
|
||||
}
|
||||
else if (implementation == "testkeeper")
|
||||
{
|
||||
@ -128,7 +131,16 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
|
||||
throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::Error::ZNONODE);
|
||||
}
|
||||
|
||||
ZooKeeper::ZooKeeper(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_,
|
||||
ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_,
|
||||
int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_)
|
||||
{
|
||||
Strings hosts_strings;
|
||||
splitInto<','>(hosts_strings, hosts_string);
|
||||
|
||||
init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_);
|
||||
}
|
||||
|
||||
ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_,
|
||||
int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_)
|
||||
{
|
||||
init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_);
|
||||
@ -141,8 +153,6 @@ struct ZooKeeperArgs
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_name, keys);
|
||||
|
||||
std::vector<std::string> hosts_strings;
|
||||
|
||||
session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS;
|
||||
operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS;
|
||||
implementation = "zookeeper";
|
||||
@ -150,7 +160,7 @@ struct ZooKeeperArgs
|
||||
{
|
||||
if (startsWith(key, "node"))
|
||||
{
|
||||
hosts_strings.push_back(
|
||||
hosts.push_back(
|
||||
(config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") +
|
||||
config.getString(config_name + "." + key + ".host") + ":"
|
||||
+ config.getString(config_name + "." + key + ".port", "2181")
|
||||
@ -180,17 +190,6 @@ struct ZooKeeperArgs
|
||||
throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS);
|
||||
}
|
||||
|
||||
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
|
||||
pcg64 generator(randomSeed());
|
||||
std::shuffle(hosts_strings.begin(), hosts_strings.end(), generator);
|
||||
|
||||
for (auto & host : hosts_strings)
|
||||
{
|
||||
if (!hosts.empty())
|
||||
hosts += ',';
|
||||
hosts += host;
|
||||
}
|
||||
|
||||
if (!chroot.empty())
|
||||
{
|
||||
if (chroot.front() != '/')
|
||||
@ -200,7 +199,7 @@ struct ZooKeeperArgs
|
||||
}
|
||||
}
|
||||
|
||||
std::string hosts;
|
||||
Strings hosts;
|
||||
std::string identity;
|
||||
int session_timeout_ms;
|
||||
int operation_timeout_ms;
|
||||
@ -922,6 +921,10 @@ Coordination::Error ZooKeeper::tryMultiNoThrow(const Coordination::Requests & re
|
||||
}
|
||||
}
|
||||
|
||||
void ZooKeeper::finalize()
|
||||
{
|
||||
impl->finalize();
|
||||
}
|
||||
|
||||
size_t KeeperMultiException::getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses)
|
||||
{
|
||||
@ -1000,4 +1003,5 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version)
|
||||
request->version = version;
|
||||
return request;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -50,7 +50,14 @@ class ZooKeeper
|
||||
public:
|
||||
using Ptr = std::shared_ptr<ZooKeeper>;
|
||||
|
||||
ZooKeeper(const std::string & hosts_, const std::string & identity_ = "",
|
||||
/// hosts_string -- comma separated [secure://]host:port list
|
||||
ZooKeeper(const std::string & hosts_string, const std::string & identity_ = "",
|
||||
int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS,
|
||||
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
|
||||
const std::string & chroot_ = "",
|
||||
const std::string & implementation_ = "zookeeper");
|
||||
|
||||
ZooKeeper(const Strings & hosts_, const std::string & identity_ = "",
|
||||
int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS,
|
||||
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
|
||||
const std::string & chroot_ = "",
|
||||
@ -247,10 +254,12 @@ public:
|
||||
/// Like the previous one but don't throw any exceptions on future.get()
|
||||
FutureMulti tryAsyncMulti(const Coordination::Requests & ops);
|
||||
|
||||
void finalize();
|
||||
|
||||
private:
|
||||
friend class EphemeralNodeHolder;
|
||||
|
||||
void init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_,
|
||||
void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_,
|
||||
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_);
|
||||
|
||||
/// The following methods don't throw exceptions but return error codes.
|
||||
@ -266,7 +275,7 @@ private:
|
||||
|
||||
std::unique_ptr<Coordination::IKeeper> impl;
|
||||
|
||||
std::string hosts;
|
||||
Strings hosts;
|
||||
std::string identity;
|
||||
int32_t session_timeout_ms;
|
||||
int32_t operation_timeout_ms;
|
||||
|
@ -88,7 +88,7 @@ using namespace DB;
|
||||
|
||||
/** Usage scenario: look at the documentation for IKeeper class.
|
||||
*/
|
||||
class ZooKeeper : public IKeeper
|
||||
class ZooKeeper final : public IKeeper
|
||||
{
|
||||
public:
|
||||
struct Node
|
||||
@ -167,6 +167,20 @@ public:
|
||||
const Requests & requests,
|
||||
MultiCallback callback) override;
|
||||
|
||||
/// Without forcefully invalidating (finalizing) ZooKeeper session before
|
||||
/// establishing a new one, there was a possibility that server is using
|
||||
/// two ZooKeeper sessions simultaneously in different parts of code.
|
||||
/// This is strong antipattern and we always prevented it.
|
||||
|
||||
/// ZooKeeper is linearizeable for writes, but not linearizeable for
|
||||
/// reads, it only maintains "sequential consistency": in every session
|
||||
/// you observe all events in order but possibly with some delay. If you
|
||||
/// perform write in one session, then notify different part of code and
|
||||
/// it will do read in another session, that read may not see the
|
||||
/// already performed write.
|
||||
|
||||
void finalize() override { finalize(false, false); }
|
||||
|
||||
private:
|
||||
String root_path;
|
||||
ACLs default_acls;
|
||||
|
@ -195,124 +195,6 @@ TEST(CoordinationTest, TestSummingRaft1)
|
||||
s1.launcher.shutdown(5);
|
||||
}
|
||||
|
||||
TEST(CoordinationTest, TestSummingRaft3)
|
||||
{
|
||||
ChangelogDirTest test1("./logs1");
|
||||
SummingRaftServer s1(1, "localhost", 44444, "./logs1");
|
||||
ChangelogDirTest test2("./logs2");
|
||||
SummingRaftServer s2(2, "localhost", 44445, "./logs2");
|
||||
ChangelogDirTest test3("./logs3");
|
||||
SummingRaftServer s3(3, "localhost", 44446, "./logs3");
|
||||
|
||||
nuraft::srv_config first_config(1, 0, "localhost:44444", "", false, 0);
|
||||
auto ret1 = s2.raft_instance->add_srv(first_config);
|
||||
while (!ret1->get_accepted())
|
||||
{
|
||||
|
||||
std::cout << "failed to add server: "
|
||||
<< ret1->get_result_str() << std::endl;
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
ret1 = s2.raft_instance->add_srv(first_config);
|
||||
}
|
||||
|
||||
while (s1.raft_instance->get_leader() != 2)
|
||||
{
|
||||
std::cout << "Waiting s1 to join to s2 quorum\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
nuraft::srv_config third_config(3, 0, "localhost:44446", "", false, 0);
|
||||
auto ret3 = s2.raft_instance->add_srv(third_config);
|
||||
if (!ret3->get_accepted())
|
||||
{
|
||||
std::cout << "failed to add server: "
|
||||
<< ret3->get_result_str() << std::endl;
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
ret3 = s2.raft_instance->add_srv(third_config);
|
||||
}
|
||||
|
||||
while (s3.raft_instance->get_leader() != 2)
|
||||
{
|
||||
std::cout << "Waiting s3 to join to s2 quorum\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
/// S2 is leader
|
||||
EXPECT_EQ(s1.raft_instance->get_leader(), 2);
|
||||
EXPECT_EQ(s2.raft_instance->get_leader(), 2);
|
||||
EXPECT_EQ(s3.raft_instance->get_leader(), 2);
|
||||
|
||||
std::cerr << "Starting to add entries\n";
|
||||
auto entry = getBuffer(1);
|
||||
auto ret = s2.raft_instance->append_entries({entry});
|
||||
while (!ret->get_accepted() || ret->get_result_code() != nuraft::cmd_result_code::OK)
|
||||
{
|
||||
std::cerr << ret->get_accepted() << "failed to replicate: entry 1" << ret->get_result_code() << std::endl;
|
||||
ret = s2.raft_instance->append_entries({entry});
|
||||
}
|
||||
|
||||
while (s1.state_machine->getValue() != 1)
|
||||
{
|
||||
std::cout << "Waiting s1 to apply entry\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
while (s2.state_machine->getValue() != 1)
|
||||
{
|
||||
std::cout << "Waiting s2 to apply entry\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
while (s3.state_machine->getValue() != 1)
|
||||
{
|
||||
std::cout << "Waiting s3 to apply entry\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
EXPECT_EQ(s1.state_machine->getValue(), 1);
|
||||
EXPECT_EQ(s2.state_machine->getValue(), 1);
|
||||
EXPECT_EQ(s3.state_machine->getValue(), 1);
|
||||
|
||||
auto non_leader_entry = getBuffer(3);
|
||||
auto ret_non_leader1 = s1.raft_instance->append_entries({non_leader_entry});
|
||||
|
||||
EXPECT_FALSE(ret_non_leader1->get_accepted());
|
||||
|
||||
auto ret_non_leader3 = s3.raft_instance->append_entries({non_leader_entry});
|
||||
|
||||
EXPECT_FALSE(ret_non_leader3->get_accepted());
|
||||
|
||||
auto leader_entry = getBuffer(77);
|
||||
auto ret_leader = s2.raft_instance->append_entries({leader_entry});
|
||||
while (!ret_leader->get_accepted() || ret_leader->get_result_code() != nuraft::cmd_result_code::OK)
|
||||
{
|
||||
std::cerr << "failed to replicate: entry 78" << ret_leader->get_result_code() << std::endl;
|
||||
ret_leader = s2.raft_instance->append_entries({leader_entry});
|
||||
}
|
||||
|
||||
while (s1.state_machine->getValue() != 78)
|
||||
{
|
||||
std::cout << "Waiting s1 to apply entry\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
while (s3.state_machine->getValue() != 78)
|
||||
{
|
||||
std::cout << "Waiting s3 to apply entry\n";
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
}
|
||||
|
||||
EXPECT_EQ(s1.state_machine->getValue(), 78);
|
||||
EXPECT_EQ(s2.state_machine->getValue(), 78);
|
||||
EXPECT_EQ(s3.state_machine->getValue(), 78);
|
||||
|
||||
s1.launcher.shutdown(5);
|
||||
s2.launcher.shutdown(5);
|
||||
s3.launcher.shutdown(5);
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
|
||||
{
|
||||
DB::WriteBufferFromNuraftBuffer buf;
|
||||
|
@ -444,6 +444,7 @@ class IColumn;
|
||||
M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
|
||||
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
|
||||
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
|
||||
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
|
||||
|
||||
// End of COMMON_SETTINGS
|
||||
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below.
|
||||
|
@ -1,12 +1,15 @@
|
||||
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -15,7 +18,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int VIOLATED_CONSTRAINT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
|
||||
@ -48,62 +51,75 @@ void CheckConstraintsBlockOutputStream::write(const Block & block)
|
||||
|
||||
ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName());
|
||||
|
||||
if (!isUInt8(res_column.type))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Constraint {} does not return a value of type UInt8",
|
||||
auto result_type = removeNullable(removeLowCardinality(res_column.type));
|
||||
|
||||
if (!isUInt8(result_type))
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Constraint {} does not return a value of type UInt8",
|
||||
backQuote(constraint_ptr->name));
|
||||
|
||||
if (const ColumnConst * res_const = typeid_cast<const ColumnConst *>(res_column.column.get()))
|
||||
{
|
||||
UInt8 value = res_const->getValue<UInt64>();
|
||||
auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
|
||||
|
||||
/// Is violated.
|
||||
if (!value)
|
||||
{
|
||||
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
|
||||
"Constraint {} for table {} is violated, because it is a constant expression returning 0. "
|
||||
"It is most likely an error in table definition.",
|
||||
backQuote(constraint_ptr->name), table_id.getNameForLogs());
|
||||
}
|
||||
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*result_column))
|
||||
{
|
||||
const auto & nested_column = column_nullable->getNestedColumnPtr();
|
||||
|
||||
/// Check if constraint value is nullable
|
||||
const auto & null_map = column_nullable->getNullMapColumn();
|
||||
const PaddedPODArray<UInt8> & data = null_map.getData();
|
||||
bool null_map_contains_null = !memoryIsZero(data.raw_data(), data.size() * sizeof(UInt8));
|
||||
|
||||
if (null_map_contains_null)
|
||||
throw Exception(
|
||||
ErrorCodes::VIOLATED_CONSTRAINT,
|
||||
"Constraint {} for table {} is violated. Expression: ({})."\
|
||||
"Constraint expression returns nullable column that contains null value",
|
||||
backQuote(constraint_ptr->name),
|
||||
table_id.getNameForLogs(),
|
||||
serializeAST(*(constraint_ptr->expr), true));
|
||||
|
||||
result_column = nested_column;
|
||||
}
|
||||
else
|
||||
|
||||
const ColumnUInt8 & res_column_uint8 = assert_cast<const ColumnUInt8 &>(*result_column);
|
||||
|
||||
const UInt8 * data = res_column_uint8.getData().data();
|
||||
size_t size = res_column_uint8.size();
|
||||
|
||||
/// Is violated.
|
||||
if (!memoryIsByte(data, size, 1))
|
||||
{
|
||||
const ColumnUInt8 & res_column_uint8 = assert_cast<const ColumnUInt8 &>(*res_column.column);
|
||||
size_t row_idx = 0;
|
||||
for (; row_idx < size; ++row_idx)
|
||||
if (data[row_idx] != 1)
|
||||
break;
|
||||
|
||||
const UInt8 * data = res_column_uint8.getData().data();
|
||||
size_t size = res_column_uint8.size();
|
||||
Names related_columns = constraint_expr->getRequiredColumns();
|
||||
|
||||
/// Is violated.
|
||||
if (!memoryIsByte(data, size, 1))
|
||||
bool first = true;
|
||||
String column_values_msg;
|
||||
constexpr size_t approx_bytes_for_col = 32;
|
||||
column_values_msg.reserve(approx_bytes_for_col * related_columns.size());
|
||||
for (const auto & name : related_columns)
|
||||
{
|
||||
size_t row_idx = 0;
|
||||
for (; row_idx < size; ++row_idx)
|
||||
if (data[row_idx] != 1)
|
||||
break;
|
||||
const IColumn & column = *block.getByName(name).column;
|
||||
assert(row_idx < column.size());
|
||||
|
||||
Names related_columns = constraint_expr->getRequiredColumns();
|
||||
|
||||
bool first = true;
|
||||
String column_values_msg;
|
||||
constexpr size_t approx_bytes_for_col = 32;
|
||||
column_values_msg.reserve(approx_bytes_for_col * related_columns.size());
|
||||
for (const auto & name : related_columns)
|
||||
{
|
||||
const IColumn & column = *block.getByName(name).column;
|
||||
assert(row_idx < column.size());
|
||||
|
||||
if (!first)
|
||||
column_values_msg.append(", ");
|
||||
column_values_msg.append(backQuoteIfNeed(name));
|
||||
column_values_msg.append(" = ");
|
||||
column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx]));
|
||||
first = false;
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
|
||||
"Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}",
|
||||
backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1,
|
||||
serializeAST(*(constraint_ptr->expr), true), column_values_msg);
|
||||
if (!first)
|
||||
column_values_msg.append(", ");
|
||||
column_values_msg.append(backQuoteIfNeed(name));
|
||||
column_values_msg.append(" = ");
|
||||
column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx]));
|
||||
first = false;
|
||||
}
|
||||
|
||||
throw Exception(
|
||||
ErrorCodes::VIOLATED_CONSTRAINT,
|
||||
"Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}",
|
||||
backQuote(constraint_ptr->name),
|
||||
table_id.getNameForLogs(),
|
||||
rows_written + row_idx + 1,
|
||||
serializeAST(*(constraint_ptr->expr), true),
|
||||
column_values_msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -47,11 +47,13 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
# include <common/logger_useful.h>
|
||||
# include <Formats/MySQLBlockInputStream.h>
|
||||
# include "readInvalidateQuery.h"
|
||||
# include <mysqlxx/Exception.h>
|
||||
# include <mysqlxx/PoolFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
static const UInt64 max_block_size = 8192;
|
||||
static const size_t default_num_tries_on_connection_loss = 3;
|
||||
|
||||
|
||||
MySQLDictionarySource::MySQLDictionarySource(
|
||||
@ -72,7 +74,10 @@ MySQLDictionarySource::MySQLDictionarySource(
|
||||
, query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks}
|
||||
, load_all_query{query_builder.composeLoadAllQuery()}
|
||||
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
|
||||
, close_connection{config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)}
|
||||
, close_connection(
|
||||
config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false))
|
||||
, max_tries_for_mysql_block_input_stream(
|
||||
config.getBool(config_prefix + ".fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss)
|
||||
{
|
||||
}
|
||||
|
||||
@ -94,6 +99,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other
|
||||
, invalidate_query{other.invalidate_query}
|
||||
, invalidate_query_response{other.invalidate_query_response}
|
||||
, close_connection{other.close_connection}
|
||||
, max_tries_for_mysql_block_input_stream{other.max_tries_for_mysql_block_input_stream}
|
||||
{
|
||||
}
|
||||
|
||||
@ -114,13 +120,41 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
|
||||
}
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MySQLDictionarySource::retriedCreateMySqlBIStream(const std::string & data_fetch_query_str, const size_t max_tries)
|
||||
{
|
||||
size_t count_connection_lost = 0;
|
||||
|
||||
while (true)
|
||||
{
|
||||
auto connection = pool.get();
|
||||
|
||||
try
|
||||
{
|
||||
return std::make_shared<MySQLBlockInputStream>(
|
||||
connection, data_fetch_query_str, sample_block, max_block_size, close_connection);
|
||||
}
|
||||
catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST
|
||||
{
|
||||
if (++count_connection_lost < max_tries)
|
||||
{
|
||||
LOG_WARNING(log, ecl.displayText());
|
||||
LOG_WARNING(log, "Lost connection ({}/{}). Trying to reconnect...", count_connection_lost, max_tries);
|
||||
continue;
|
||||
}
|
||||
|
||||
LOG_ERROR(log, "Failed ({}/{}) to create BlockInputStream for MySQL dictionary source.", count_connection_lost, max_tries);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MySQLDictionarySource::loadAll()
|
||||
{
|
||||
auto connection = pool.get();
|
||||
last_modification = getLastModification(connection, false);
|
||||
|
||||
LOG_TRACE(log, load_all_query);
|
||||
return std::make_shared<MySQLBlockInputStream>(connection, load_all_query, sample_block, max_block_size, close_connection);
|
||||
return retriedCreateMySqlBIStream(load_all_query, max_tries_for_mysql_block_input_stream);
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
|
||||
@ -130,7 +164,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
|
||||
|
||||
std::string load_update_query = getUpdateFieldAndDate();
|
||||
LOG_TRACE(log, load_update_query);
|
||||
return std::make_shared<MySQLBlockInputStream>(connection, load_update_query, sample_block, max_block_size, close_connection);
|
||||
return retriedCreateMySqlBIStream(load_update_query, max_tries_for_mysql_block_input_stream);
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
@ -138,7 +172,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & i
|
||||
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
||||
|
||||
const auto query = query_builder.composeLoadIdsQuery(ids);
|
||||
return std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size, close_connection);
|
||||
return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream);
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
||||
@ -146,7 +180,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns,
|
||||
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
||||
|
||||
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
|
||||
return std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size, close_connection);
|
||||
return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream);
|
||||
}
|
||||
|
||||
bool MySQLDictionarySource::isModified() const
|
||||
|
@ -69,6 +69,9 @@ private:
|
||||
// execute invalidate_query. expects single cell in result
|
||||
std::string doInvalidateQuery(const std::string & request) const;
|
||||
|
||||
/// A helper method for recovering from "Lost connection to MySQL server during query" errors
|
||||
BlockInputStreamPtr retriedCreateMySqlBIStream(const std::string & query_str, const size_t max_tries);
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> update_time;
|
||||
@ -86,6 +89,7 @@ private:
|
||||
std::string invalidate_query;
|
||||
mutable std::string invalidate_query_response;
|
||||
const bool close_connection;
|
||||
const size_t max_tries_for_mysql_block_input_stream;
|
||||
};
|
||||
|
||||
}
|
||||
|
358
src/Functions/extractTextFromHTML.cpp
Normal file
358
src/Functions/extractTextFromHTML.cpp
Normal file
@ -0,0 +1,358 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
|
||||
/** A function to extract text from HTML or XHTML.
|
||||
* It does not necessarily 100% conforms to any of the HTML, XML or XHTML standards,
|
||||
* but the implementation is reasonably accurate and it is fast.
|
||||
*
|
||||
* The rules are the following:
|
||||
*
|
||||
* 1. Comments are skipped. Example: <!-- test -->
|
||||
* Comment must end with -->. Nested comments are not possible.
|
||||
* Note: constructions like <!--> <!---> are not valid comments in HTML but will be skipped by other rules.
|
||||
*
|
||||
* 2. CDATA is pasted verbatim.
|
||||
* Note: CDATA is XML/XHTML specific. But we still process it for "best-effort" approach.
|
||||
*
|
||||
* 3. 'script' and 'style' elements are removed with all their content.
|
||||
* Note: it's assumed that closing tag cannot appear inside content.
|
||||
* For example, in JS string literal is has to be escaped as "<\/script>".
|
||||
* Note: comments and CDATA is possible inside script or style - then closing tags are not searched inside CDATA.
|
||||
* Example: <script><![CDATA[</script>]]></script>
|
||||
* But still searched inside comments. Sometimes it becomes complicated:
|
||||
* <script>var x = "<!--"; </script> var y = "-->"; alert(x + y);</script>
|
||||
* Note: script and style can be the names of XML namespaces - then they are not treat like usual script or style.
|
||||
* Example: <script:a>Hello</script:a>.
|
||||
* Note: whitespaces are possible after closing tag name: </script > but not before: < / script>.
|
||||
*
|
||||
* 4. Other tags or tag-like elements are skipped without inner content.
|
||||
* Example: <a>.</a>
|
||||
* Note: it's expected that this HTML is illegal: <a test=">"></a>
|
||||
* Note: it will also skip something like tags: <>, <!>, etc.
|
||||
* Note: tag without end will be skipped to the end of input: <hello
|
||||
* >
|
||||
* 5. HTML and XML entities are not decoded.
|
||||
* It should be processed by separate function.
|
||||
*
|
||||
* 6. Whitespaces in text are collapsed or inserted by specific rules.
|
||||
* Whitespaces at beginning and at the end are removed.
|
||||
* Consecutive whitespaces are collapsed.
|
||||
* But if text is separated by other elements and there is no whitespace, it is inserted.
|
||||
* It may be unnatural, examples: Hello<b>world</b>, Hello<!-- -->world
|
||||
* - in HTML there will be no whitespace, but the function will insert it.
|
||||
* But also consider: Hello<p>world</p>, Hello<br>world.
|
||||
* This behaviour is reasonable for data analysis, e.g. convert HTML to a bag of words.
|
||||
*
|
||||
* 7. Also note that correct handling of whitespaces would require
|
||||
* support of <pre></pre> and CSS display and white-space properties.
|
||||
*
|
||||
* Usage example:
|
||||
*
|
||||
* SELECT extractTextFromHTML(html) FROM url('https://yandex.ru/', RawBLOB, 'html String')
|
||||
*
|
||||
* - ClickHouse has embedded web browser.
|
||||
*/
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
inline bool startsWith(const char * s, const char * end, const char * prefix)
|
||||
{
|
||||
return s + strlen(prefix) < end && 0 == memcmp(s, prefix, strlen(prefix));
|
||||
}
|
||||
|
||||
inline bool checkAndSkip(const char * __restrict & s, const char * end, const char * prefix)
|
||||
{
|
||||
if (startsWith(s, end, prefix))
|
||||
{
|
||||
s += strlen(prefix);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool processComment(const char * __restrict & src, const char * end)
|
||||
{
|
||||
if (!checkAndSkip(src, end, "<!--"))
|
||||
return false;
|
||||
|
||||
while (true)
|
||||
{
|
||||
const char * gt = find_first_symbols<'>'>(src, end);
|
||||
if (gt >= end)
|
||||
break;
|
||||
|
||||
if (gt > src + strlen("--") && gt[-1] == '-' && gt[-2] == '-')
|
||||
{
|
||||
src = gt + 1;
|
||||
break;
|
||||
}
|
||||
|
||||
src = gt + 1;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool processCDATA(const char * __restrict & src, const char * end, char * __restrict & dst)
|
||||
{
|
||||
if (!checkAndSkip(src, end, "<![CDATA["))
|
||||
return false;
|
||||
|
||||
const char * gt = src;
|
||||
while (true)
|
||||
{
|
||||
gt = find_first_symbols<'>'>(gt, end);
|
||||
if (gt >= end)
|
||||
break;
|
||||
|
||||
if (gt[-1] == ']' && gt[-2] == ']')
|
||||
{
|
||||
if (dst)
|
||||
{
|
||||
size_t bytes_to_copy = gt - src - strlen("]]");
|
||||
memcpy(dst, src, bytes_to_copy);
|
||||
dst += bytes_to_copy;
|
||||
}
|
||||
src = gt + 1;
|
||||
break;
|
||||
}
|
||||
|
||||
++gt;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool processElementAndSkipContent(const char * __restrict & src, const char * end, const char * tag_name)
|
||||
{
|
||||
const auto * old_src = src;
|
||||
|
||||
if (!(src < end && *src == '<'))
|
||||
return false;
|
||||
++src;
|
||||
|
||||
if (!checkAndSkip(src, end, tag_name))
|
||||
{
|
||||
src = old_src;
|
||||
return false;
|
||||
}
|
||||
|
||||
if (src >= end)
|
||||
return false;
|
||||
|
||||
if (!(isWhitespaceASCII(*src) || *src == '>'))
|
||||
{
|
||||
src = old_src;
|
||||
return false;
|
||||
}
|
||||
|
||||
const char * gt = find_first_symbols<'>'>(src, end);
|
||||
if (gt >= end)
|
||||
return false;
|
||||
|
||||
src = gt + 1;
|
||||
|
||||
while (true)
|
||||
{
|
||||
const char * lt = find_first_symbols<'<'>(src, end);
|
||||
src = lt;
|
||||
if (src + 1 >= end)
|
||||
break;
|
||||
|
||||
++src;
|
||||
|
||||
/// Skip CDATA
|
||||
if (*src == '!')
|
||||
{
|
||||
--src;
|
||||
char * dst = nullptr;
|
||||
if (processCDATA(src, end, dst))
|
||||
continue;
|
||||
++src;
|
||||
}
|
||||
|
||||
if (*src != '/')
|
||||
continue;
|
||||
++src;
|
||||
|
||||
if (checkAndSkip(src, end, tag_name))
|
||||
{
|
||||
while (src < end && isWhitespaceASCII(*src))
|
||||
++src;
|
||||
|
||||
if (src >= end)
|
||||
break;
|
||||
|
||||
if (*src == '>')
|
||||
{
|
||||
++src;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool skipTag(const char * __restrict & src, const char * end)
|
||||
{
|
||||
if (src < end && *src == '<')
|
||||
{
|
||||
src = find_first_symbols<'>'>(src, end);
|
||||
if (src < end)
|
||||
++src;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void copyText(const char * __restrict & src, const char * end, char * __restrict & dst, bool needs_whitespace)
|
||||
{
|
||||
while (src < end && isWhitespaceASCII(*src))
|
||||
++src;
|
||||
|
||||
const char * lt = find_first_symbols<'<'>(src, end);
|
||||
|
||||
if (needs_whitespace && src < lt)
|
||||
{
|
||||
*dst = ' ';
|
||||
++dst;
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
const char * ws = find_first_symbols<' ', '\t', '\n', '\r', '\f', '\v'>(src, lt);
|
||||
size_t bytes_to_copy = ws - src;
|
||||
memcpy(dst, src, bytes_to_copy);
|
||||
dst += bytes_to_copy;
|
||||
|
||||
src = ws;
|
||||
while (src < lt && isWhitespaceASCII(*src))
|
||||
++src;
|
||||
|
||||
if (src < lt)
|
||||
{
|
||||
*dst = ' ';
|
||||
++dst;
|
||||
}
|
||||
else
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
src = lt;
|
||||
}
|
||||
|
||||
size_t extract(const char * __restrict src, size_t size, char * __restrict dst)
|
||||
{
|
||||
/** There are the following rules:
|
||||
* - comments are removed with all their content;
|
||||
* - elements 'script' and 'style' are removed with all their content;
|
||||
* - for other elements tags are removed but content is processed as text;
|
||||
* - CDATA should be copied verbatim;
|
||||
*/
|
||||
|
||||
const char * end = src + size;
|
||||
char * dst_begin = dst;
|
||||
|
||||
while (src < end)
|
||||
{
|
||||
bool needs_whitespace = dst != dst_begin && dst[-1] != ' ';
|
||||
copyText(src, end, dst, needs_whitespace);
|
||||
|
||||
processComment(src, end)
|
||||
|| processCDATA(src, end, dst)
|
||||
|| processElementAndSkipContent(src, end, "script")
|
||||
|| processElementAndSkipContent(src, end, "style")
|
||||
|| skipTag(src, end);
|
||||
}
|
||||
|
||||
return dst - dst_begin;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
class FunctionExtractTextFromHTML : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "extractTextFromHTML";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionExtractTextFromHTML>(); }
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
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);
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t rows) const override
|
||||
{
|
||||
const ColumnString * src = checkAndGetColumn<ColumnString>(arguments[0].column.get());
|
||||
if (!src)
|
||||
throw Exception("First argument for function " + getName() + " must be string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const ColumnString::Chars & src_chars = src->getChars();
|
||||
const ColumnString::Offsets & src_offsets = src->getOffsets();
|
||||
|
||||
auto res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & res_chars = res->getChars();
|
||||
ColumnString::Offsets & res_offsets = res->getOffsets();
|
||||
|
||||
res_chars.resize(src_chars.size());
|
||||
res_offsets.resize(src_offsets.size());
|
||||
|
||||
ColumnString::Offset src_offset = 0;
|
||||
ColumnString::Offset res_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
auto next_src_offset = src_offsets[i];
|
||||
|
||||
res_offset += extract(
|
||||
reinterpret_cast<const char *>(&src_chars[src_offset]),
|
||||
next_src_offset - src_offset - 1,
|
||||
reinterpret_cast<char *>(&res_chars[res_offset]));
|
||||
|
||||
res_chars[res_offset] = 0;
|
||||
++res_offset;
|
||||
res_offsets[i] = res_offset;
|
||||
|
||||
src_offset = next_src_offset;
|
||||
}
|
||||
|
||||
res_chars.resize(res_offset);
|
||||
return res;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionExtractTextFromHTML(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionExtractTextFromHTML>();
|
||||
}
|
||||
|
||||
}
|
@ -1,582 +0,0 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
#include <algorithm>
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
# include <hs.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
struct HxCoarseParseImpl
|
||||
{
|
||||
private:
|
||||
struct SpanInfo
|
||||
{
|
||||
SpanInfo(): id(0), match_space(std::pair<unsigned long long, unsigned long long>(0, 0)) {} // NOLINT
|
||||
SpanInfo(unsigned int matchId, std::pair<unsigned long long, unsigned long long> matchSpan): id(matchId), match_space(matchSpan){} // NOLINT
|
||||
SpanInfo(const SpanInfo& obj)
|
||||
{
|
||||
id = obj.id;
|
||||
match_space = obj.match_space;
|
||||
}
|
||||
SpanInfo& operator=(const SpanInfo& obj) = default;
|
||||
|
||||
unsigned int id;
|
||||
std::pair<unsigned long long, unsigned long long> match_space; // NOLINT
|
||||
};
|
||||
using SpanElement = std::vector<SpanInfo>;
|
||||
struct Span
|
||||
{
|
||||
Span(): set_script(false), set_style(false), set_semi(false), is_finding_cdata(false) {}
|
||||
|
||||
SpanElement copy_stack; // copy area
|
||||
SpanElement tag_stack; // regexp area
|
||||
SpanInfo script_ptr; // script pointer
|
||||
bool set_script; // whether set script
|
||||
SpanInfo style_ptr; // style pointer
|
||||
bool set_style; // whether set style
|
||||
SpanInfo semi_ptr; // tag ptr
|
||||
bool set_semi; // whether set semi
|
||||
|
||||
bool is_finding_cdata;
|
||||
};
|
||||
|
||||
static inline void copyZone(
|
||||
ColumnString::Offset& current_dst_string_offset,
|
||||
ColumnString::Offset& current_copy_loc,
|
||||
ColumnString::Chars& dst_chars,
|
||||
const ColumnString::Chars& src_chars,
|
||||
size_t bytes_to_copy,
|
||||
unsigned is_space
|
||||
)
|
||||
{
|
||||
bool is_last_space = false;
|
||||
if (current_dst_string_offset == 0 || dst_chars[current_dst_string_offset - 1] == 0 || dst_chars[current_dst_string_offset - 1] == ' ')
|
||||
{
|
||||
is_last_space = true;
|
||||
}
|
||||
if (bytes_to_copy == 0)
|
||||
{
|
||||
if (is_space && !is_last_space)
|
||||
{
|
||||
dst_chars[current_dst_string_offset++] = ' ';
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_last_space && src_chars[current_copy_loc] == ' ')
|
||||
{
|
||||
--bytes_to_copy;
|
||||
++current_copy_loc;
|
||||
}
|
||||
if (bytes_to_copy > 0)
|
||||
{
|
||||
memcpySmallAllowReadWriteOverflow15(
|
||||
&dst_chars[current_dst_string_offset], &src_chars[current_copy_loc], bytes_to_copy);
|
||||
current_dst_string_offset += bytes_to_copy;
|
||||
}
|
||||
|
||||
// separator is space and last character is not space.
|
||||
if (is_space && !(current_dst_string_offset == 0 || dst_chars[current_dst_string_offset - 1] == 0 || dst_chars[current_dst_string_offset - 1] == ' '))
|
||||
{
|
||||
dst_chars[current_dst_string_offset++] = ' ';
|
||||
}
|
||||
}
|
||||
// return;
|
||||
}
|
||||
static inline void popArea(SpanElement& stack, unsigned long long from, unsigned long long to) //NOLINT
|
||||
{
|
||||
while (!stack.empty())
|
||||
{
|
||||
if (to > stack.back().match_space.second && from < stack.back().match_space.second)
|
||||
{
|
||||
stack.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
// return;
|
||||
}
|
||||
|
||||
static void dealCommonTag(Span* matches)
|
||||
{
|
||||
while (!matches->copy_stack.empty() && matches->copy_stack.back().id != 10)
|
||||
{
|
||||
matches->copy_stack.pop_back();
|
||||
}
|
||||
if (!matches->copy_stack.empty())
|
||||
{
|
||||
matches->copy_stack.pop_back();
|
||||
}
|
||||
unsigned long long from; // NOLINT
|
||||
unsigned long long to; // NOLINT
|
||||
unsigned id;
|
||||
for (auto begin = matches->tag_stack.begin(); begin != matches->tag_stack.end(); ++begin)
|
||||
{
|
||||
from = begin->match_space.first;
|
||||
to = begin->match_space.second;
|
||||
id = begin->id;
|
||||
switch (id)
|
||||
{
|
||||
case 12:
|
||||
case 13:
|
||||
{
|
||||
popArea(matches->copy_stack, from, to);
|
||||
if (matches->copy_stack.empty() || from >= matches->copy_stack.back().match_space.second)
|
||||
matches->copy_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
break;
|
||||
}
|
||||
case 0:
|
||||
case 2:
|
||||
case 3:
|
||||
case 4:
|
||||
case 5:
|
||||
case 6:
|
||||
case 7:
|
||||
case 8:
|
||||
case 9:
|
||||
case 10:
|
||||
{
|
||||
if (!matches->set_semi || (matches->set_semi && from == matches->semi_ptr.match_space.first))
|
||||
{
|
||||
matches->set_semi = true;
|
||||
matches->semi_ptr = SpanInfo(id, std::make_pair(from, to));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 1:
|
||||
{
|
||||
if (matches->set_semi)
|
||||
{
|
||||
switch (matches->semi_ptr.id)
|
||||
{
|
||||
case 0:
|
||||
case 2:
|
||||
case 3:
|
||||
case 6:
|
||||
case 7:
|
||||
case 10:
|
||||
{
|
||||
if (matches->semi_ptr.id == 2 || (matches->semi_ptr.id == 3 && matches->semi_ptr.match_space.second == from))
|
||||
{
|
||||
if (!matches->set_script)
|
||||
{
|
||||
matches->set_script = true;
|
||||
matches->script_ptr = SpanInfo(matches->semi_ptr.id, std::make_pair(matches->semi_ptr.match_space.first, to));
|
||||
}
|
||||
}
|
||||
else if (matches->semi_ptr.id == 6 || (matches->semi_ptr.id == 7 && matches->semi_ptr.match_space.second == from))
|
||||
{
|
||||
if (!matches->set_style)
|
||||
{
|
||||
matches->set_style = true;
|
||||
matches->style_ptr = SpanInfo(matches->semi_ptr.id, std::make_pair(matches->semi_ptr.match_space.first, to));
|
||||
}
|
||||
}
|
||||
popArea(matches->copy_stack, matches->semi_ptr.match_space.first, to);
|
||||
matches->copy_stack.push_back(SpanInfo(0, std::make_pair(matches->semi_ptr.match_space.first, to)));
|
||||
matches->set_semi = false;
|
||||
break;
|
||||
}
|
||||
case 4:
|
||||
case 5:
|
||||
case 8:
|
||||
case 9:
|
||||
{
|
||||
SpanInfo complete_zone;
|
||||
|
||||
complete_zone.match_space.second = to;
|
||||
if (matches->set_script && (matches->semi_ptr.id == 4 || (matches->semi_ptr.id == 5 && matches->semi_ptr.match_space.second == from)))
|
||||
{
|
||||
complete_zone.id = matches->script_ptr.id;
|
||||
complete_zone.match_space.first = matches->script_ptr.match_space.first;
|
||||
matches->set_script = false;
|
||||
}
|
||||
else if (matches->set_style && (matches->semi_ptr.id == 8 || (matches->semi_ptr.id == 9 && matches->semi_ptr.match_space.second == from)))
|
||||
{
|
||||
complete_zone.id = matches->style_ptr.id;
|
||||
complete_zone.match_space.first = matches->style_ptr.match_space.first;
|
||||
matches->set_style = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
complete_zone.id = matches->semi_ptr.id;
|
||||
complete_zone.match_space.first = matches->semi_ptr.match_space.first;
|
||||
}
|
||||
popArea(matches->copy_stack, complete_zone.match_space.first, complete_zone.match_space.second);
|
||||
matches->copy_stack.push_back(complete_zone);
|
||||
matches->set_semi = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
// return;
|
||||
}
|
||||
static int spanCollect(unsigned int id,
|
||||
unsigned long long from, // NOLINT
|
||||
unsigned long long to, // NOLINT
|
||||
unsigned int , void * ctx)
|
||||
{
|
||||
Span* matches = static_cast<Span*>(ctx);
|
||||
from = id == 12 ? from : to - patterns_length[id];
|
||||
|
||||
if (matches->is_finding_cdata)
|
||||
{
|
||||
if (id == 11)
|
||||
{
|
||||
matches->copy_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
matches->is_finding_cdata = false;
|
||||
matches->tag_stack.clear();
|
||||
if (matches->semi_ptr.id == 10)
|
||||
{
|
||||
matches->set_semi = false;
|
||||
}
|
||||
}
|
||||
else if (id == 12 || id == 13)
|
||||
{
|
||||
popArea(matches->copy_stack, from, to);
|
||||
if (matches->copy_stack.empty() || from >= matches->copy_stack.back().match_space.second)
|
||||
matches->copy_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
|
||||
popArea(matches->tag_stack, from, to);
|
||||
if (matches->tag_stack.empty() || from >= matches->tag_stack.back().match_space.second)
|
||||
matches->tag_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
}
|
||||
else
|
||||
{
|
||||
popArea(matches->tag_stack, from, to);
|
||||
matches->tag_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
switch (id)
|
||||
{
|
||||
case 12:
|
||||
case 13:
|
||||
{
|
||||
popArea(matches->copy_stack, from, to);
|
||||
if (matches->copy_stack.empty() || from >= matches->copy_stack.back().match_space.second)
|
||||
matches->copy_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
break;
|
||||
}
|
||||
case 0:
|
||||
case 2:
|
||||
case 3:
|
||||
case 4:
|
||||
case 5:
|
||||
case 6:
|
||||
case 7:
|
||||
case 8:
|
||||
case 9:
|
||||
{
|
||||
if (!matches->set_semi || (matches->set_semi && from == matches->semi_ptr.match_space.first))
|
||||
{
|
||||
matches->set_semi = true;
|
||||
matches->semi_ptr = SpanInfo(id, std::make_pair(from, to));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 10:
|
||||
{
|
||||
if (!matches->set_semi || (matches->set_semi && from == matches->semi_ptr.match_space.first))
|
||||
{
|
||||
matches->set_semi = true;
|
||||
matches->semi_ptr = SpanInfo(id, std::make_pair(from, to));
|
||||
}
|
||||
matches->is_finding_cdata = true;
|
||||
matches->copy_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
matches->tag_stack.push_back(SpanInfo(id, std::make_pair(from, to)));
|
||||
break;
|
||||
}
|
||||
case 1:
|
||||
{
|
||||
if (matches->set_semi)
|
||||
{
|
||||
switch (matches->semi_ptr.id)
|
||||
{
|
||||
case 0:
|
||||
case 2:
|
||||
case 3:
|
||||
case 6:
|
||||
case 7:
|
||||
case 10:
|
||||
{
|
||||
if (matches->semi_ptr.id == 2 || (matches->semi_ptr.id == 3 && matches->semi_ptr.match_space.second == from))
|
||||
{
|
||||
if (!matches->set_script)
|
||||
{
|
||||
matches->set_script = true;
|
||||
matches->script_ptr = SpanInfo(matches->semi_ptr.id, std::make_pair(matches->semi_ptr.match_space.first, to));
|
||||
}
|
||||
}
|
||||
else if (matches->semi_ptr.id == 6 || (matches->semi_ptr.id == 7 && matches->semi_ptr.match_space.second == from))
|
||||
{
|
||||
if (!matches->set_style)
|
||||
{
|
||||
matches->set_style = true;
|
||||
matches->style_ptr = SpanInfo(matches->semi_ptr.id, std::make_pair(matches->semi_ptr.match_space.first, to));
|
||||
}
|
||||
}
|
||||
popArea(matches->copy_stack, matches->semi_ptr.match_space.first, to);
|
||||
matches->copy_stack.push_back(SpanInfo(matches->semi_ptr.id, std::make_pair(matches->semi_ptr.match_space.first, to)));
|
||||
matches->set_semi = false;
|
||||
break;
|
||||
}
|
||||
case 4:
|
||||
case 5:
|
||||
case 8:
|
||||
case 9:
|
||||
{
|
||||
SpanInfo complete_zone;
|
||||
complete_zone.match_space.second = to;
|
||||
if (matches->set_script && (matches->semi_ptr.id == 4 || (matches->semi_ptr.id == 5 && matches->semi_ptr.match_space.second == from)))
|
||||
{
|
||||
complete_zone.id = matches->script_ptr.id;
|
||||
complete_zone.match_space.first = matches->script_ptr.match_space.first;
|
||||
matches->set_script = false;
|
||||
}
|
||||
else if (matches->set_style && (matches->semi_ptr.id == 8 || (matches->semi_ptr.id == 9 && matches->semi_ptr.match_space.second == from)))
|
||||
{
|
||||
complete_zone.id = matches->style_ptr.id;
|
||||
complete_zone.match_space.first = matches->style_ptr.match_space.first;
|
||||
matches->set_style = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
complete_zone.id = matches->semi_ptr.id;
|
||||
complete_zone.match_space.first = matches->semi_ptr.match_space.first;
|
||||
}
|
||||
popArea(matches->copy_stack, complete_zone.match_space.first, complete_zone.match_space.second);
|
||||
matches->copy_stack.push_back(complete_zone);
|
||||
matches->set_semi = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
#if USE_HYPERSCAN
|
||||
static hs_database_t* buildDatabase(const std::vector<const char* > &expressions,
|
||||
const std::vector<unsigned> &flags,
|
||||
const std::vector<unsigned> &id,
|
||||
unsigned int mode)
|
||||
{
|
||||
hs_database_t *db;
|
||||
hs_compile_error_t *compile_err;
|
||||
hs_error_t err;
|
||||
err = hs_compile_multi(expressions.data(), flags.data(), id.data(),
|
||||
expressions.size(), mode, nullptr, &db, &compile_err);
|
||||
|
||||
if (err != HS_SUCCESS)
|
||||
{
|
||||
hs_free_compile_error(compile_err);
|
||||
throw Exception("Hyper scan database cannot be compiled.", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
return db;
|
||||
}
|
||||
#endif
|
||||
static std::vector<const char*> patterns;
|
||||
static std::vector<std::size_t> patterns_length;
|
||||
static std::vector<unsigned> patterns_flag;
|
||||
static std::vector<unsigned> ids;
|
||||
|
||||
public:
|
||||
static void executeInternal(
|
||||
const ColumnString::Chars & src_chars,
|
||||
const ColumnString::Offsets & src_offsets,
|
||||
ColumnString::Chars & dst_chars,
|
||||
ColumnString::Offsets & dst_offsets)
|
||||
{
|
||||
#if USE_HYPERSCAN
|
||||
hs_database_t * db = buildDatabase(patterns, patterns_flag, ids, HS_MODE_BLOCK);
|
||||
hs_scratch_t* scratch = nullptr;
|
||||
if (hs_alloc_scratch(db, &scratch) != HS_SUCCESS)
|
||||
{
|
||||
hs_free_database(db);
|
||||
throw Exception("Unable to allocate scratch space.", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
dst_chars.resize(src_chars.size());
|
||||
dst_offsets.resize(src_offsets.size());
|
||||
|
||||
ColumnString::Offset current_src_string_offset = 0;
|
||||
ColumnString::Offset current_dst_string_offset = 0;
|
||||
ColumnString::Offset current_copy_loc;
|
||||
ColumnString::Offset current_copy_end;
|
||||
unsigned is_space;
|
||||
size_t bytes_to_copy;
|
||||
Span match_zoneall;
|
||||
|
||||
for (size_t off = 0; off < src_offsets.size(); ++off)
|
||||
{
|
||||
hs_scan(db, reinterpret_cast<const char *>(&src_chars[current_src_string_offset]), src_offsets[off] - current_src_string_offset, 0, scratch, spanCollect, &match_zoneall);
|
||||
if (match_zoneall.is_finding_cdata)
|
||||
{
|
||||
dealCommonTag(&match_zoneall);
|
||||
}
|
||||
SpanElement& match_zone = match_zoneall.copy_stack;
|
||||
current_copy_loc = current_src_string_offset;
|
||||
if (match_zone.empty())
|
||||
{
|
||||
current_copy_end = src_offsets[off];
|
||||
is_space = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
current_copy_end = current_src_string_offset + match_zone.begin()->match_space.first;
|
||||
is_space = (match_zone.begin()->id == 12 || match_zone.begin()->id == 13)?1:0;
|
||||
}
|
||||
|
||||
bytes_to_copy = current_copy_end - current_copy_loc;
|
||||
copyZone(current_dst_string_offset, current_copy_loc, dst_chars, src_chars, bytes_to_copy, is_space);
|
||||
for (auto begin = match_zone.begin(); begin != match_zone.end(); ++begin)
|
||||
{
|
||||
current_copy_loc = current_src_string_offset + begin->match_space.second;
|
||||
if (begin + 1 >= match_zone.end())
|
||||
{
|
||||
current_copy_end = src_offsets[off];
|
||||
is_space = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
current_copy_end = current_src_string_offset + (begin+1)->match_space.first;
|
||||
is_space = ((begin+1)->id == 12 || (begin+1)->id == 13)?1:0;
|
||||
}
|
||||
bytes_to_copy = current_copy_end - current_copy_loc;
|
||||
copyZone(current_dst_string_offset, current_copy_loc, dst_chars, src_chars, bytes_to_copy, is_space);
|
||||
}
|
||||
if (current_dst_string_offset > 1 && dst_chars[current_dst_string_offset - 2] == ' ')
|
||||
{
|
||||
dst_chars[current_dst_string_offset - 2] = 0;
|
||||
--current_dst_string_offset;
|
||||
}
|
||||
dst_offsets[off] = current_dst_string_offset;
|
||||
current_src_string_offset = src_offsets[off];
|
||||
match_zoneall.copy_stack.clear();
|
||||
match_zoneall.tag_stack.clear();
|
||||
}
|
||||
dst_chars.resize(dst_chars.size());
|
||||
hs_free_scratch(scratch);
|
||||
hs_free_database(db);
|
||||
#else
|
||||
(void)src_chars;
|
||||
(void)src_offsets;
|
||||
(void)dst_chars;
|
||||
(void)dst_offsets;
|
||||
throw Exception(
|
||||
"htmlOrXmlCoarseParse is not implemented when hyperscan is off (is it x86 processor?)",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
};
|
||||
|
||||
std::vector<const char*> HxCoarseParseImpl::patterns =
|
||||
{
|
||||
"<[^\\s<>]", // 0 "<", except "< ", "<<", "<>"
|
||||
">", // 1 ">"
|
||||
"<script\\s", // 2 <script xxxxx>
|
||||
"<script", // 3 <script>
|
||||
"</script\\s", // 4 </script xxxx>
|
||||
"</script", // 5 </script>
|
||||
"<style\\s", // 6 <style xxxxxx>
|
||||
"<style", // 7 <style>
|
||||
"</style\\s", // 8 </style xxxxx>
|
||||
"</style", // 9 </style>
|
||||
"<!\\[CDATA\\[", // 10 <![CDATA[xxxxxx]]>
|
||||
"\\]\\]>", // 11 ]]>
|
||||
"\\s{2,}", // 12 " ", continuous blanks
|
||||
"[^\\S ]" // 13 "\n", "\t" and other white space, it does not include single ' '.
|
||||
};
|
||||
std::vector<std::size_t> HxCoarseParseImpl::patterns_length =
|
||||
{
|
||||
2, 1, 8, 7, 9, 8, 7, 6, 8, 7, 9, 3, 0, 1
|
||||
};
|
||||
#if USE_HYPERSCAN
|
||||
std::vector<unsigned> HxCoarseParseImpl::patterns_flag =
|
||||
{
|
||||
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, HS_FLAG_SOM_LEFTMOST, 0
|
||||
};
|
||||
#endif
|
||||
std::vector<unsigned> HxCoarseParseImpl::ids =
|
||||
{
|
||||
0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13
|
||||
};
|
||||
|
||||
class FunctionHtmlOrXmlCoarseParse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "htmlOrXmlCoarseParse";
|
||||
|
||||
static FunctionPtr create(const Context &) {return std::make_shared<FunctionHtmlOrXmlCoarseParse>(); }
|
||||
|
||||
String getName() const override {return name;}
|
||||
|
||||
size_t getNumberOfArguments() 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);
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override {return true;}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & , size_t) const override
|
||||
{
|
||||
const auto & strcolumn = arguments[0].column;
|
||||
if (const ColumnString* html_sentence = checkAndGetColumn<ColumnString>(strcolumn.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
HxCoarseParseImpl::executeInternal(html_sentence->getChars(), html_sentence->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("First argument for function " + getName() + " must be string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
void registerFunctionHtmlOrXmlCoarseParse(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionHtmlOrXmlCoarseParse>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
@ -6,9 +6,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class FunctionFactory;
|
||||
#if USE_HYPERSCAN
|
||||
void registerFunctionHtmlOrXmlCoarseParse(FunctionFactory &);
|
||||
#endif
|
||||
|
||||
void registerFunctionRepeat(FunctionFactory &);
|
||||
void registerFunctionEmpty(FunctionFactory &);
|
||||
void registerFunctionNotEmpty(FunctionFactory &);
|
||||
@ -35,8 +33,9 @@ void registerFunctionRegexpQuoteMeta(FunctionFactory &);
|
||||
void registerFunctionNormalizeQuery(FunctionFactory &);
|
||||
void registerFunctionNormalizedQueryHash(FunctionFactory &);
|
||||
void registerFunctionCountMatches(FunctionFactory &);
|
||||
void registerFunctionEncodeXMLComponent(FunctionFactory & factory);
|
||||
void registerFunctionDecodeXMLComponent(FunctionFactory & factory);
|
||||
void registerFunctionEncodeXMLComponent(FunctionFactory &);
|
||||
void registerFunctionDecodeXMLComponent(FunctionFactory &);
|
||||
void registerFunctionExtractTextFromHTML(FunctionFactory &);
|
||||
|
||||
|
||||
#if USE_BASE64
|
||||
@ -47,9 +46,6 @@ void registerFunctionTryBase64Decode(FunctionFactory &);
|
||||
|
||||
void registerFunctionsString(FunctionFactory & factory)
|
||||
{
|
||||
#if USE_HYPERSCAN
|
||||
registerFunctionHtmlOrXmlCoarseParse(factory);
|
||||
#endif
|
||||
registerFunctionRepeat(factory);
|
||||
registerFunctionEmpty(factory);
|
||||
registerFunctionNotEmpty(factory);
|
||||
@ -78,6 +74,7 @@ void registerFunctionsString(FunctionFactory & factory)
|
||||
registerFunctionCountMatches(factory);
|
||||
registerFunctionEncodeXMLComponent(factory);
|
||||
registerFunctionDecodeXMLComponent(factory);
|
||||
registerFunctionExtractTextFromHTML(factory);
|
||||
#if USE_BASE64
|
||||
registerFunctionBase64Encode(factory);
|
||||
registerFunctionBase64Decode(factory);
|
||||
|
@ -246,6 +246,7 @@ SRCS(
|
||||
extractAllGroupsHorizontal.cpp
|
||||
extractAllGroupsVertical.cpp
|
||||
extractGroups.cpp
|
||||
extractTextFromHTML.cpp
|
||||
extractTimeZoneFromFunctionArguments.cpp
|
||||
filesystem.cpp
|
||||
finalizeAggregation.cpp
|
||||
@ -291,7 +292,6 @@ SRCS(
|
||||
hasToken.cpp
|
||||
hasTokenCaseInsensitive.cpp
|
||||
hostName.cpp
|
||||
htmlOrXmlCoarseParse.cpp
|
||||
hypot.cpp
|
||||
identity.cpp
|
||||
if.cpp
|
||||
|
@ -1,71 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Writes data asynchronously using double buffering.
|
||||
*/
|
||||
class AsynchronousWriteBuffer : public WriteBuffer
|
||||
{
|
||||
private:
|
||||
WriteBuffer & out; /// The main buffer, responsible for writing data.
|
||||
std::vector <char> memory; /// A piece of memory for duplicating the buffer.
|
||||
ThreadPool pool; /// For asynchronous data writing.
|
||||
bool started; /// Has an asynchronous data write started?
|
||||
|
||||
/// Swap the main and duplicate buffers.
|
||||
void swapBuffers()
|
||||
{
|
||||
swap(out);
|
||||
}
|
||||
|
||||
void nextImpl() override
|
||||
{
|
||||
if (!offset())
|
||||
return;
|
||||
|
||||
if (started)
|
||||
pool.wait();
|
||||
else
|
||||
started = true;
|
||||
|
||||
swapBuffers();
|
||||
|
||||
/// The data will be written in separate stream.
|
||||
pool.scheduleOrThrowOnError([this] { thread(); });
|
||||
}
|
||||
|
||||
public:
|
||||
AsynchronousWriteBuffer(WriteBuffer & out_) : WriteBuffer(nullptr, 0), out(out_), memory(out.buffer().size()), pool(1), started(false)
|
||||
{
|
||||
/// Data is written to the duplicate buffer.
|
||||
set(memory.data(), memory.size());
|
||||
}
|
||||
|
||||
~AsynchronousWriteBuffer() override
|
||||
{
|
||||
/// FIXME move final flush into the caller
|
||||
MemoryTracker::LockExceptionInThread lock;
|
||||
|
||||
if (started)
|
||||
pool.wait();
|
||||
|
||||
swapBuffers();
|
||||
out.next();
|
||||
}
|
||||
|
||||
/// That is executed in a separate thread
|
||||
void thread()
|
||||
{
|
||||
out.next();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -64,29 +64,38 @@ void BrotliWriteBuffer::nextImpl()
|
||||
in_data = reinterpret_cast<unsigned char *>(working_buffer.begin());
|
||||
in_available = offset();
|
||||
|
||||
do
|
||||
try
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
out_data = reinterpret_cast<unsigned char *>(out->position());
|
||||
out_capacity = out->buffer().end() - out->position();
|
||||
|
||||
int result = BrotliEncoderCompressStream(
|
||||
brotli->state,
|
||||
in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
|
||||
&in_available,
|
||||
&in_data,
|
||||
&out_capacity,
|
||||
&out_data,
|
||||
nullptr);
|
||||
|
||||
out->position() = out->buffer().end() - out_capacity;
|
||||
|
||||
if (result == 0)
|
||||
do
|
||||
{
|
||||
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
|
||||
out->nextIfAtEnd();
|
||||
out_data = reinterpret_cast<unsigned char *>(out->position());
|
||||
out_capacity = out->buffer().end() - out->position();
|
||||
|
||||
int result = BrotliEncoderCompressStream(
|
||||
brotli->state,
|
||||
in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
|
||||
&in_available,
|
||||
&in_data,
|
||||
&out_capacity,
|
||||
&out_data,
|
||||
nullptr);
|
||||
|
||||
out->position() = out->buffer().end() - out_capacity;
|
||||
|
||||
if (result == 0)
|
||||
{
|
||||
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
|
||||
}
|
||||
}
|
||||
while (in_available > 0);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to write next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
throw;
|
||||
}
|
||||
while (in_available > 0 || out_capacity == 0);
|
||||
}
|
||||
|
||||
void BrotliWriteBuffer::finish()
|
||||
@ -94,6 +103,23 @@ void BrotliWriteBuffer::finish()
|
||||
if (finished)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
finishImpl();
|
||||
out->next();
|
||||
finished = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to flush next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
finished = true;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void BrotliWriteBuffer::finishImpl()
|
||||
{
|
||||
next();
|
||||
|
||||
while (true)
|
||||
@ -115,7 +141,6 @@ void BrotliWriteBuffer::finish()
|
||||
|
||||
if (BrotliEncoderIsFinished(brotli->state))
|
||||
{
|
||||
finished = true;
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -18,11 +18,14 @@ public:
|
||||
|
||||
~BrotliWriteBuffer() override;
|
||||
|
||||
void finish();
|
||||
void finalize() override { finish(); }
|
||||
|
||||
private:
|
||||
void nextImpl() override;
|
||||
|
||||
void finish();
|
||||
void finishImpl();
|
||||
|
||||
class BrotliStateWrapper;
|
||||
std::unique_ptr<BrotliStateWrapper> brotli;
|
||||
|
||||
|
@ -1,30 +0,0 @@
|
||||
#include <common/types.h>
|
||||
#include <Common/hex.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <IO/HexWriteBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void HexWriteBuffer::nextImpl()
|
||||
{
|
||||
if (!offset())
|
||||
return;
|
||||
|
||||
for (Position p = working_buffer.begin(); p != pos; ++p)
|
||||
{
|
||||
UInt8 byte = *p;
|
||||
out.write(hexDigitUppercase(byte / 16));
|
||||
out.write(hexDigitUppercase(byte % 16));
|
||||
}
|
||||
}
|
||||
|
||||
HexWriteBuffer::~HexWriteBuffer()
|
||||
{
|
||||
/// FIXME move final flush into the caller
|
||||
MemoryTracker::LockExceptionInThread lock;
|
||||
nextImpl();
|
||||
}
|
||||
|
||||
}
|
@ -1,28 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
|
||||
/// Since HexWriteBuffer is often created in the inner loop, we'll make its buffer size small.
|
||||
#define DBMS_HEX_WRITE_BUFFER_SIZE 32
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Everything that is written into it, translates to HEX (in capital letters) and writes to another WriteBuffer.
|
||||
*/
|
||||
class HexWriteBuffer final : public WriteBuffer
|
||||
{
|
||||
protected:
|
||||
char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; //-V730
|
||||
WriteBuffer & out;
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
public:
|
||||
HexWriteBuffer(WriteBuffer & out_) : WriteBuffer(buf, sizeof(buf)), out(out_) {}
|
||||
~HexWriteBuffer() override;
|
||||
};
|
||||
|
||||
}
|
@ -64,27 +64,36 @@ void LZMADeflatingWriteBuffer::nextImpl()
|
||||
lstr.next_in = reinterpret_cast<unsigned char *>(working_buffer.begin());
|
||||
lstr.avail_in = offset();
|
||||
|
||||
lzma_action action = LZMA_RUN;
|
||||
do
|
||||
try
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
lstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
lstr.avail_out = out->buffer().end() - out->position();
|
||||
lzma_action action = LZMA_RUN;
|
||||
do
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
lstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
lstr.avail_out = out->buffer().end() - out->position();
|
||||
|
||||
lzma_ret ret = lzma_code(&lstr, action);
|
||||
out->position() = out->buffer().end() - lstr.avail_out;
|
||||
lzma_ret ret = lzma_code(&lstr, action);
|
||||
out->position() = out->buffer().end() - lstr.avail_out;
|
||||
|
||||
if (ret == LZMA_STREAM_END)
|
||||
return;
|
||||
if (ret == LZMA_STREAM_END)
|
||||
return;
|
||||
|
||||
if (ret != LZMA_OK)
|
||||
throw Exception(
|
||||
ErrorCodes::LZMA_STREAM_ENCODER_FAILED,
|
||||
"lzma stream encoding failed: error code: {}; lzma_version: {}",
|
||||
ret,
|
||||
LZMA_VERSION_STRING);
|
||||
if (ret != LZMA_OK)
|
||||
throw Exception(
|
||||
ErrorCodes::LZMA_STREAM_ENCODER_FAILED,
|
||||
"lzma stream encoding failed: error code: {}; lzma_version: {}",
|
||||
ret,
|
||||
LZMA_VERSION_STRING);
|
||||
|
||||
} while (lstr.avail_in > 0 || lstr.avail_out == 0);
|
||||
} while (lstr.avail_in > 0 || lstr.avail_out == 0);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to write next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -93,6 +102,23 @@ void LZMADeflatingWriteBuffer::finish()
|
||||
if (finished)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
finishImpl();
|
||||
out->next();
|
||||
finished = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to flush next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
finished = true;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void LZMADeflatingWriteBuffer::finishImpl()
|
||||
{
|
||||
next();
|
||||
|
||||
do
|
||||
@ -106,7 +132,6 @@ void LZMADeflatingWriteBuffer::finish()
|
||||
|
||||
if (ret == LZMA_STREAM_END)
|
||||
{
|
||||
finished = true;
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -24,13 +24,16 @@ public:
|
||||
char * existing_memory = nullptr,
|
||||
size_t alignment = 0);
|
||||
|
||||
void finish();
|
||||
void finalize() override { finish(); }
|
||||
|
||||
~LZMADeflatingWriteBuffer() override;
|
||||
|
||||
private:
|
||||
void nextImpl() override;
|
||||
|
||||
void finish();
|
||||
void finishImpl();
|
||||
|
||||
std::unique_ptr<WriteBuffer> out;
|
||||
lzma_stream lstr;
|
||||
bool finished = false;
|
||||
|
@ -75,19 +75,28 @@ void ZlibDeflatingWriteBuffer::nextImpl()
|
||||
zstr.next_in = reinterpret_cast<unsigned char *>(working_buffer.begin());
|
||||
zstr.avail_in = offset();
|
||||
|
||||
do
|
||||
try
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
zstr.avail_out = out->buffer().end() - out->position();
|
||||
do
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
zstr.avail_out = out->buffer().end() - out->position();
|
||||
|
||||
int rc = deflate(&zstr, Z_NO_FLUSH);
|
||||
out->position() = out->buffer().end() - zstr.avail_out;
|
||||
int rc = deflate(&zstr, Z_NO_FLUSH);
|
||||
out->position() = out->buffer().end() - zstr.avail_out;
|
||||
|
||||
if (rc != Z_OK)
|
||||
throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
|
||||
if (rc != Z_OK)
|
||||
throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
|
||||
}
|
||||
while (zstr.avail_in > 0 || zstr.avail_out == 0);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to write next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
throw;
|
||||
}
|
||||
while (zstr.avail_in > 0 || zstr.avail_out == 0);
|
||||
}
|
||||
|
||||
void ZlibDeflatingWriteBuffer::finish()
|
||||
@ -95,6 +104,23 @@ void ZlibDeflatingWriteBuffer::finish()
|
||||
if (finished)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
finishImpl();
|
||||
out->next();
|
||||
finished = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to flush next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
finished = true;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void ZlibDeflatingWriteBuffer::finishImpl()
|
||||
{
|
||||
next();
|
||||
|
||||
/// https://github.com/zlib-ng/zlib-ng/issues/494
|
||||
@ -123,7 +149,6 @@ void ZlibDeflatingWriteBuffer::finish()
|
||||
|
||||
if (rc == Z_STREAM_END)
|
||||
{
|
||||
finished = true;
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -22,16 +22,19 @@ public:
|
||||
char * existing_memory = nullptr,
|
||||
size_t alignment = 0);
|
||||
|
||||
/// Flush all pending data and write zlib footer to the underlying buffer.
|
||||
/// After the first call to this function, subsequent calls will have no effect and
|
||||
/// an attempt to write to this buffer will result in exception.
|
||||
void finish();
|
||||
void finalize() override { finish(); }
|
||||
|
||||
~ZlibDeflatingWriteBuffer() override;
|
||||
|
||||
private:
|
||||
void nextImpl() override;
|
||||
|
||||
void finishImpl();
|
||||
/// Flush all pending data and write zlib footer to the underlying buffer.
|
||||
/// After the first call to this function, subsequent calls will have no effect and
|
||||
/// an attempt to write to this buffer will result in exception.
|
||||
void finish();
|
||||
|
||||
std::unique_ptr<WriteBuffer> out;
|
||||
z_stream zstr;
|
||||
bool finished = false;
|
||||
|
@ -61,28 +61,53 @@ void ZstdDeflatingWriteBuffer::nextImpl()
|
||||
input.size = offset();
|
||||
input.pos = 0;
|
||||
|
||||
bool finished = false;
|
||||
do
|
||||
try
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
bool ended = false;
|
||||
do
|
||||
{
|
||||
out->nextIfAtEnd();
|
||||
|
||||
output.dst = reinterpret_cast<unsigned char *>(out->buffer().begin());
|
||||
output.size = out->buffer().size();
|
||||
output.pos = out->offset();
|
||||
output.dst = reinterpret_cast<unsigned char *>(out->buffer().begin());
|
||||
output.size = out->buffer().size();
|
||||
output.pos = out->offset();
|
||||
|
||||
|
||||
ZSTD_compressStream2(cctx, &output, &input, mode);
|
||||
out->position() = out->buffer().begin() + output.pos;
|
||||
finished = (input.pos == input.size);
|
||||
} while (!finished);
|
||||
|
||||
ZSTD_compressStream2(cctx, &output, &input, mode);
|
||||
out->position() = out->buffer().begin() + output.pos;
|
||||
ended = (input.pos == input.size);
|
||||
} while (!ended);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to write next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void ZstdDeflatingWriteBuffer::finish()
|
||||
{
|
||||
if (flushed)
|
||||
if (finished)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
finishImpl();
|
||||
out->next();
|
||||
finished = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Do not try to flush next time after exception.
|
||||
out->position() = out->buffer().begin();
|
||||
finished = true;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void ZstdDeflatingWriteBuffer::finishImpl()
|
||||
{
|
||||
next();
|
||||
|
||||
out->nextIfAtEnd();
|
||||
@ -99,7 +124,6 @@ void ZstdDeflatingWriteBuffer::finish()
|
||||
if (ZSTD_isError(remaining))
|
||||
throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder end failed: zstd version: {}", ZSTD_VERSION_STRING);
|
||||
out->position() = out->buffer().begin() + output.pos;
|
||||
flushed = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,21 +20,24 @@ public:
|
||||
char * existing_memory = nullptr,
|
||||
size_t alignment = 0);
|
||||
|
||||
/// Flush all pending data and write zstd footer to the underlying buffer.
|
||||
/// After the first call to this function, subsequent calls will have no effect and
|
||||
/// an attempt to write to this buffer will result in exception.
|
||||
void finish();
|
||||
void finalize() override { finish(); }
|
||||
|
||||
~ZstdDeflatingWriteBuffer() override;
|
||||
|
||||
private:
|
||||
void nextImpl() override;
|
||||
|
||||
/// Flush all pending data and write zstd footer to the underlying buffer.
|
||||
/// After the first call to this function, subsequent calls will have no effect and
|
||||
/// an attempt to write to this buffer will result in exception.
|
||||
void finish();
|
||||
void finishImpl();
|
||||
|
||||
std::unique_ptr<WriteBuffer> out;
|
||||
ZSTD_CCtx * cctx;
|
||||
ZSTD_inBuffer input;
|
||||
ZSTD_outBuffer output;
|
||||
bool flushed = false;
|
||||
bool finished = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -25,9 +25,6 @@ target_link_libraries (var_uint PRIVATE clickhouse_common_io)
|
||||
add_executable (read_escaped_string read_escaped_string.cpp)
|
||||
target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io)
|
||||
|
||||
add_executable (async_write async_write.cpp)
|
||||
target_link_libraries (async_write PRIVATE dbms)
|
||||
|
||||
add_executable (parse_int_perf parse_int_perf.cpp)
|
||||
target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io)
|
||||
|
||||
|
@ -1,26 +0,0 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/AsynchronousWriteBuffer.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
|
||||
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
|
||||
DB::WriteBufferFromFileDescriptor out1(STDOUT_FILENO);
|
||||
DB::AsynchronousWriteBuffer out2(out1);
|
||||
DB::CompressedWriteBuffer out3(out2);
|
||||
|
||||
DB::copyData(in1, out3);
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
}
|
@ -28,7 +28,7 @@ try
|
||||
DB::writeIntText(i, lzma_buf);
|
||||
DB::writeChar('\t', lzma_buf);
|
||||
}
|
||||
lzma_buf.finish();
|
||||
lzma_buf.finalize();
|
||||
|
||||
stopwatch.stop();
|
||||
|
||||
|
@ -62,7 +62,6 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
DB::WriteBufferFromVector wb(formatted);
|
||||
// DB::CompressedWriteBuffer wb2(wb1);
|
||||
// DB::AsynchronousWriteBuffer wb(wb2);
|
||||
Stopwatch watch;
|
||||
|
||||
UInt64 tsc = rdtsc();
|
||||
|
@ -30,7 +30,7 @@ try
|
||||
DB::writeIntText(i, deflating_buf);
|
||||
DB::writeChar('\t', deflating_buf);
|
||||
}
|
||||
deflating_buf.finish();
|
||||
deflating_buf.finalize();
|
||||
|
||||
stopwatch.stop();
|
||||
std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s."
|
||||
|
@ -30,7 +30,7 @@ try
|
||||
DB::writeIntText(i, zstd_buf);
|
||||
DB::writeChar('\t', zstd_buf);
|
||||
}
|
||||
zstd_buf.finish();
|
||||
zstd_buf.finalize();
|
||||
|
||||
stopwatch.stop();
|
||||
|
||||
|
@ -29,7 +29,6 @@ SRCS(
|
||||
HTTPChunkedReadBuffer.cpp
|
||||
HTTPCommon.cpp
|
||||
HashingWriteBuffer.cpp
|
||||
HexWriteBuffer.cpp
|
||||
LZMADeflatingWriteBuffer.cpp
|
||||
LZMAInflatingReadBuffer.cpp
|
||||
LimitReadBuffer.cpp
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/materialize.h>
|
||||
#include <Functions/FunctionsLogical.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionJIT.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
@ -364,7 +365,7 @@ void ActionsDAG::removeUnusedActions(const std::vector<Node *> & required_nodes)
|
||||
removeUnusedActions();
|
||||
}
|
||||
|
||||
void ActionsDAG::removeUnusedActions()
|
||||
void ActionsDAG::removeUnusedActions(bool allow_remove_inputs)
|
||||
{
|
||||
std::unordered_set<const Node *> visited_nodes;
|
||||
std::stack<Node *> stack;
|
||||
@ -388,6 +389,9 @@ void ActionsDAG::removeUnusedActions()
|
||||
visited_nodes.insert(&node);
|
||||
stack.push(&node);
|
||||
}
|
||||
|
||||
if (node.type == ActionType::INPUT && !allow_remove_inputs)
|
||||
visited_nodes.insert(&node);
|
||||
}
|
||||
|
||||
while (!stack.empty())
|
||||
@ -516,6 +520,11 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name)
|
||||
if (col == child)
|
||||
return false;
|
||||
|
||||
/// Do not remove input if it was mentioned in index several times.
|
||||
for (const auto * node : index)
|
||||
if (col == node)
|
||||
return false;
|
||||
|
||||
/// Remove from nodes and inputs.
|
||||
for (auto jt = nodes.begin(); jt != nodes.end(); ++jt)
|
||||
{
|
||||
@ -1203,4 +1212,340 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co
|
||||
return split(split_nodes);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
struct ConjunctionNodes
|
||||
{
|
||||
std::vector<ActionsDAG::Node *> allowed;
|
||||
std::vector<ActionsDAG::Node *> rejected;
|
||||
};
|
||||
|
||||
/// Take a node which result is predicate.
|
||||
/// Assuming predicate is a conjunction (probably, trivial).
|
||||
/// Find separate conjunctions nodes. Split nodes into allowed and rejected sets.
|
||||
/// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set.
|
||||
ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordered_set<const ActionsDAG::Node *> allowed_nodes)
|
||||
{
|
||||
ConjunctionNodes conjunction;
|
||||
std::unordered_set<ActionsDAG::Node *> allowed;
|
||||
std::unordered_set<ActionsDAG::Node *> rejected;
|
||||
|
||||
struct Frame
|
||||
{
|
||||
ActionsDAG::Node * node;
|
||||
bool is_predicate = false;
|
||||
size_t next_child_to_visit = 0;
|
||||
size_t num_allowed_children = 0;
|
||||
};
|
||||
|
||||
std::stack<Frame> stack;
|
||||
std::unordered_set<ActionsDAG::Node *> visited_nodes;
|
||||
|
||||
stack.push(Frame{.node = predicate, .is_predicate = true});
|
||||
visited_nodes.insert(predicate);
|
||||
while (!stack.empty())
|
||||
{
|
||||
auto & cur = stack.top();
|
||||
bool is_conjunction = cur.is_predicate
|
||||
&& cur.node->type == ActionsDAG::ActionType::FUNCTION
|
||||
&& cur.node->function_base->getName() == "and";
|
||||
|
||||
/// At first, visit all children.
|
||||
while (cur.next_child_to_visit < cur.node->children.size())
|
||||
{
|
||||
auto * child = cur.node->children[cur.next_child_to_visit];
|
||||
|
||||
if (visited_nodes.count(child) == 0)
|
||||
{
|
||||
visited_nodes.insert(child);
|
||||
stack.push({.node = child, .is_predicate = is_conjunction});
|
||||
break;
|
||||
}
|
||||
|
||||
if (allowed_nodes.contains(child))
|
||||
++cur.num_allowed_children;
|
||||
++cur.next_child_to_visit;
|
||||
}
|
||||
|
||||
if (cur.next_child_to_visit == cur.node->children.size())
|
||||
{
|
||||
if (cur.num_allowed_children == cur.node->children.size())
|
||||
{
|
||||
if (cur.node->type != ActionsDAG::ActionType::ARRAY_JOIN && cur.node->type != ActionsDAG::ActionType::INPUT)
|
||||
allowed_nodes.emplace(cur.node);
|
||||
}
|
||||
else if (is_conjunction)
|
||||
{
|
||||
for (auto * child : cur.node->children)
|
||||
{
|
||||
if (allowed_nodes.count(child))
|
||||
{
|
||||
if (allowed.insert(child).second)
|
||||
conjunction.allowed.push_back(child);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (cur.is_predicate)
|
||||
{
|
||||
if (rejected.insert(cur.node).second)
|
||||
conjunction.rejected.push_back(cur.node);
|
||||
}
|
||||
|
||||
stack.pop();
|
||||
}
|
||||
}
|
||||
|
||||
if (conjunction.allowed.empty())
|
||||
{
|
||||
/// If nothing was added to conjunction, check if it is trivial.
|
||||
if (allowed_nodes.count(predicate))
|
||||
conjunction.allowed.push_back(predicate);
|
||||
}
|
||||
|
||||
return conjunction;
|
||||
}
|
||||
|
||||
ColumnsWithTypeAndName prepareFunctionArguments(const std::vector<ActionsDAG::Node *> nodes)
|
||||
{
|
||||
ColumnsWithTypeAndName arguments;
|
||||
arguments.reserve(nodes.size());
|
||||
|
||||
for (const auto * child : nodes)
|
||||
{
|
||||
ColumnWithTypeAndName argument;
|
||||
argument.column = child->column;
|
||||
argument.type = child->result_type;
|
||||
argument.name = child->result_name;
|
||||
|
||||
arguments.emplace_back(std::move(argument));
|
||||
}
|
||||
|
||||
return arguments;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// Create actions which calculate conjunction of selected nodes.
|
||||
/// Assume conjunction nodes are predicates (and may be used as arguments of function AND).
|
||||
///
|
||||
/// Result actions add single column with conjunction result (it is always last in index).
|
||||
/// No other columns are added or removed.
|
||||
ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(std::vector<Node *> conjunction)
|
||||
{
|
||||
if (conjunction.empty())
|
||||
return nullptr;
|
||||
|
||||
auto actions = cloneEmpty();
|
||||
actions->settings.project_input = false;
|
||||
|
||||
FunctionOverloadResolverPtr func_builder_and =
|
||||
std::make_shared<FunctionOverloadResolverAdaptor>(
|
||||
std::make_unique<DefaultOverloadResolver>(
|
||||
std::make_shared<FunctionAnd>()));
|
||||
|
||||
std::unordered_map<const ActionsDAG::Node *, ActionsDAG::Node *> nodes_mapping;
|
||||
|
||||
struct Frame
|
||||
{
|
||||
const ActionsDAG::Node * node;
|
||||
size_t next_child_to_visit = 0;
|
||||
};
|
||||
|
||||
std::stack<Frame> stack;
|
||||
|
||||
/// DFS. Clone actions.
|
||||
for (const auto * predicate : conjunction)
|
||||
{
|
||||
if (nodes_mapping.count(predicate))
|
||||
continue;
|
||||
|
||||
stack.push({.node = predicate});
|
||||
while (!stack.empty())
|
||||
{
|
||||
auto & cur = stack.top();
|
||||
/// At first, visit all children.
|
||||
while (cur.next_child_to_visit < cur.node->children.size())
|
||||
{
|
||||
auto * child = cur.node->children[cur.next_child_to_visit];
|
||||
|
||||
if (nodes_mapping.count(child) == 0)
|
||||
{
|
||||
stack.push({.node = child});
|
||||
break;
|
||||
}
|
||||
|
||||
++cur.next_child_to_visit;
|
||||
}
|
||||
|
||||
if (cur.next_child_to_visit == cur.node->children.size())
|
||||
{
|
||||
auto & node = actions->nodes.emplace_back(*cur.node);
|
||||
nodes_mapping[cur.node] = &node;
|
||||
|
||||
for (auto & child : node.children)
|
||||
child = nodes_mapping[child];
|
||||
|
||||
if (node.type == ActionType::INPUT)
|
||||
{
|
||||
actions->inputs.emplace_back(&node);
|
||||
actions->index.insert(&node);
|
||||
}
|
||||
|
||||
stack.pop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Node * result_predicate = nodes_mapping[*conjunction.begin()];
|
||||
|
||||
if (conjunction.size() > 1)
|
||||
{
|
||||
std::vector<Node *> args;
|
||||
args.reserve(conjunction.size());
|
||||
for (const auto * predicate : conjunction)
|
||||
args.emplace_back(nodes_mapping[predicate]);
|
||||
|
||||
result_predicate = &actions->addFunction(func_builder_and, args, {}, true, false);
|
||||
}
|
||||
|
||||
actions->index.insert(result_predicate);
|
||||
return actions;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs)
|
||||
{
|
||||
Node * predicate;
|
||||
|
||||
{
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == filter_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Index for ActionsDAG does not contain filter column name {}. DAG:\n{}",
|
||||
filter_name, dumpDAG());
|
||||
|
||||
predicate = *it;
|
||||
}
|
||||
|
||||
std::unordered_set<const Node *> allowed_nodes;
|
||||
|
||||
/// Get input nodes from available_inputs names.
|
||||
{
|
||||
std::unordered_map<std::string_view, std::list<const Node *>> inputs_map;
|
||||
for (const auto & input : inputs)
|
||||
inputs_map[input->result_name].emplace_back(input);
|
||||
|
||||
for (const auto & name : available_inputs)
|
||||
{
|
||||
auto & inputs_list = inputs_map[name];
|
||||
if (inputs_list.empty())
|
||||
continue;
|
||||
|
||||
allowed_nodes.emplace(inputs_list.front());
|
||||
inputs_list.pop_front();
|
||||
}
|
||||
}
|
||||
|
||||
auto conjunction = getConjunctionNodes(predicate, allowed_nodes);
|
||||
auto actions = cloneActionsForConjunction(conjunction.allowed);
|
||||
if (!actions)
|
||||
return nullptr;
|
||||
|
||||
/// Now, when actions are created, update current DAG.
|
||||
|
||||
if (conjunction.rejected.empty())
|
||||
{
|
||||
/// The whole predicate was split.
|
||||
if (can_remove_filter)
|
||||
{
|
||||
/// If filter column is not needed, remove it from index.
|
||||
for (auto i = index.begin(); i != index.end(); ++i)
|
||||
{
|
||||
if (*i == predicate)
|
||||
{
|
||||
index.remove(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Replace predicate result to constant 1.
|
||||
Node node;
|
||||
node.type = ActionType::COLUMN;
|
||||
node.result_name = std::move(predicate->result_name);
|
||||
node.result_type = std::move(predicate->result_type);
|
||||
node.column = node.result_type->createColumnConst(0, 1);
|
||||
*predicate = std::move(node);
|
||||
}
|
||||
|
||||
removeUnusedActions(false);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Predicate is conjunction, where both allowed and rejected sets are not empty.
|
||||
/// Replace this node to conjunction of rejected predicates.
|
||||
|
||||
std::vector<Node *> new_children(conjunction.rejected.begin(), conjunction.rejected.end());
|
||||
|
||||
if (new_children.size() == 1)
|
||||
{
|
||||
/// Rejected set has only one predicate.
|
||||
if (new_children.front()->result_type->equals(*predicate->result_type))
|
||||
{
|
||||
/// If it's type is same, just add alias.
|
||||
Node node;
|
||||
node.type = ActionType::ALIAS;
|
||||
node.result_name = predicate->result_name;
|
||||
node.result_type = predicate->result_type;
|
||||
node.children.swap(new_children);
|
||||
*predicate = std::move(node);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If type is different, cast column.
|
||||
/// This case is possible, cause AND can use any numeric type as argument.
|
||||
Node node;
|
||||
node.type = ActionType::COLUMN;
|
||||
node.result_name = predicate->result_type->getName();
|
||||
node.column = DataTypeString().createColumnConst(0, node.result_name);
|
||||
node.result_type = std::make_shared<DataTypeString>();
|
||||
|
||||
auto * right_arg = &nodes.emplace_back(std::move(node));
|
||||
auto * left_arg = new_children.front();
|
||||
|
||||
predicate->children = {left_arg, right_arg};
|
||||
auto arguments = prepareFunctionArguments(predicate->children);
|
||||
|
||||
FunctionOverloadResolverPtr func_builder_cast =
|
||||
std::make_shared<FunctionOverloadResolverAdaptor>(
|
||||
CastOverloadResolver<CastType::nonAccurate>::createImpl(false));
|
||||
|
||||
predicate->function_builder = func_builder_cast;
|
||||
predicate->function_base = predicate->function_builder->build(arguments);
|
||||
predicate->function = predicate->function_base->prepare(arguments);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Predicate is function AND, which still have more then one argument.
|
||||
/// Just update children and rebuild it.
|
||||
predicate->children.swap(new_children);
|
||||
auto arguments = prepareFunctionArguments(predicate->children);
|
||||
|
||||
predicate->function_base = predicate->function_builder->build(arguments);
|
||||
predicate->function = predicate->function_base->prepare(arguments);
|
||||
}
|
||||
|
||||
removeUnusedActions(false);
|
||||
}
|
||||
|
||||
return actions;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -152,6 +152,9 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
/// NOTE: std::list is an implementation detail.
|
||||
/// It allows to add and remove new nodes inplace without reallocation.
|
||||
/// Raw pointers to nodes remain valid.
|
||||
using Nodes = std::list<Node>;
|
||||
using Inputs = std::vector<Node *>;
|
||||
|
||||
@ -278,6 +281,13 @@ public:
|
||||
/// Index of initial actions must contain column_name.
|
||||
SplitResult splitActionsForFilter(const std::string & column_name) const;
|
||||
|
||||
/// Create actions which may calculate part of filter using only available_inputs.
|
||||
/// If nothing may be calculated, returns nullptr.
|
||||
/// Otherwise, return actions which inputs are from available_inputs.
|
||||
/// Returned actions add single column which may be used for filter.
|
||||
/// Also, replace some nodes of current inputs to constant 1 in case they are filtered.
|
||||
ActionsDAGPtr splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs);
|
||||
|
||||
private:
|
||||
Node & addNode(Node node, bool can_replace = false, bool add_to_index = true);
|
||||
Node & getNode(const std::string & name);
|
||||
@ -302,10 +312,12 @@ private:
|
||||
}
|
||||
|
||||
void removeUnusedActions(const std::vector<Node *> & required_nodes);
|
||||
void removeUnusedActions();
|
||||
void removeUnusedActions(bool allow_remove_inputs = true);
|
||||
void addAliases(const NamesWithAliases & aliases, std::vector<Node *> & result_nodes);
|
||||
|
||||
void compileFunctions();
|
||||
|
||||
ActionsDAGPtr cloneActionsForConjunction(std::vector<Node *> conjunction);
|
||||
};
|
||||
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/HexWriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -284,7 +284,7 @@ void SelectStreamFactory::createForShard(
|
||||
if (try_results.empty() || local_delay < max_remote_delay)
|
||||
{
|
||||
auto plan = createLocalPlan(modified_query_ast, header, context, stage);
|
||||
return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline()));
|
||||
return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()))));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -1661,7 +1661,12 @@ void Context::resetZooKeeper() const
|
||||
static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk)
|
||||
{
|
||||
if (!zk || zk->configChanged(*config, config_name))
|
||||
{
|
||||
if (zk)
|
||||
zk->finalize();
|
||||
|
||||
zk = std::make_shared<zkutil::ZooKeeper>(*config, config_name);
|
||||
}
|
||||
}
|
||||
|
||||
void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const
|
||||
|
@ -21,7 +21,7 @@
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -122,8 +122,10 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
|
||||
try
|
||||
{
|
||||
PullingPipelineExecutor executor(io.pipeline);
|
||||
if (!executor.pull(block))
|
||||
PullingAsyncPipelineExecutor executor(io.pipeline);
|
||||
while (block.rows() == 0 && executor.pull(block));
|
||||
|
||||
if (block.rows() == 0)
|
||||
{
|
||||
/// Interpret subquery with empty result as Null literal
|
||||
auto ast_new = std::make_unique<ASTLiteral>(Null());
|
||||
@ -132,7 +134,13 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
return;
|
||||
}
|
||||
|
||||
if (block.rows() != 1 || executor.pull(block))
|
||||
if (block.rows() != 1)
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
|
||||
Block tmp_block;
|
||||
while (tmp_block.rows() == 0 && executor.pull(tmp_block));
|
||||
|
||||
if (tmp_block.rows() != 0)
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
|
@ -54,7 +54,7 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
|
||||
namespace DB
|
||||
@ -320,7 +320,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr
|
||||
|
||||
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options);
|
||||
auto io = interpreter_subquery->execute();
|
||||
PullingPipelineExecutor executor(io.pipeline);
|
||||
PullingAsyncPipelineExecutor executor(io.pipeline);
|
||||
|
||||
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in);
|
||||
set->setHeader(executor.getHeader());
|
||||
@ -328,6 +328,9 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr
|
||||
Block block;
|
||||
while (executor.pull(block))
|
||||
{
|
||||
if (block.rows() == 0)
|
||||
continue;
|
||||
|
||||
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
|
||||
if (!set->insertFromBlock(block))
|
||||
return;
|
||||
|
@ -1311,7 +1311,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
|
||||
|
||||
void HashJoin::joinTotals(Block & block) const
|
||||
{
|
||||
JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, key_names_right, block);
|
||||
JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, *table_join, block);
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,7 +28,9 @@ public:
|
||||
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0;
|
||||
|
||||
virtual bool hasTotals() const = 0;
|
||||
/// Set totals for right table
|
||||
virtual void setTotals(const Block & block) = 0;
|
||||
/// Add totals to block from left table
|
||||
virtual void joinTotals(Block & block) const = 0;
|
||||
|
||||
virtual size_t getTotalRowCount() const = 0;
|
||||
|
@ -117,7 +117,7 @@ struct QueryPlanSettings
|
||||
{
|
||||
QueryPlan::ExplainPlanOptions query_plan_options;
|
||||
|
||||
/// Apply query plan optimisations.
|
||||
/// Apply query plan optimizations.
|
||||
bool optimize = true;
|
||||
|
||||
constexpr static char name[] = "PLAN";
|
||||
@ -251,7 +251,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
interpreter.buildQueryPlan(plan);
|
||||
|
||||
if (settings.optimize)
|
||||
plan.optimize();
|
||||
plan.optimize(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
|
||||
plan.explainPlan(buf, settings.query_plan_options);
|
||||
}
|
||||
@ -265,7 +265,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
auto pipeline = plan.buildQueryPipeline();
|
||||
auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
|
||||
if (settings.graph)
|
||||
{
|
||||
|
@ -548,7 +548,7 @@ BlockIO InterpreterSelectQuery::execute()
|
||||
|
||||
buildQueryPlan(query_plan);
|
||||
|
||||
res.pipeline = std::move(*query_plan.buildQueryPipeline());
|
||||
res.pipeline = std::move(*query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef())));
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -413,7 +413,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute()
|
||||
QueryPlan query_plan;
|
||||
buildQueryPlan(query_plan);
|
||||
|
||||
auto pipeline = query_plan.buildQueryPipeline();
|
||||
auto pipeline = query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef()));
|
||||
|
||||
res.pipeline = std::move(*pipeline);
|
||||
res.pipeline.addInterpreterContext(context);
|
||||
|
@ -496,7 +496,7 @@ void MergeJoin::setTotals(const Block & totals_block)
|
||||
|
||||
void MergeJoin::joinTotals(Block & block) const
|
||||
{
|
||||
JoinCommon::joinTotals(totals, right_columns_to_add, table_join->keyNamesRight(), block);
|
||||
JoinCommon::joinTotals(totals, right_columns_to_add, *table_join, block);
|
||||
}
|
||||
|
||||
void MergeJoin::mergeRightBlocks()
|
||||
|
@ -756,7 +756,7 @@ QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vecto
|
||||
}
|
||||
}
|
||||
|
||||
auto pipeline = plan.buildQueryPipeline();
|
||||
auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
pipeline->addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<MaterializingTransform>(header);
|
||||
|
@ -251,13 +251,23 @@ void createMissedColumns(Block & block)
|
||||
}
|
||||
}
|
||||
|
||||
void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block)
|
||||
/// Append totals from right to left block, correct types if needed
|
||||
void joinTotals(const Block & totals, const Block & columns_to_add, const TableJoin & table_join, Block & block)
|
||||
{
|
||||
if (table_join.forceNullableLeft())
|
||||
convertColumnsToNullable(block);
|
||||
|
||||
if (Block totals_without_keys = totals)
|
||||
{
|
||||
for (const auto & name : key_names_right)
|
||||
for (const auto & name : table_join.keyNamesRight())
|
||||
totals_without_keys.erase(totals_without_keys.getPositionByName(name));
|
||||
|
||||
for (auto & col : totals_without_keys)
|
||||
{
|
||||
if (table_join.rightBecomeNullable(col.type))
|
||||
JoinCommon::convertColumnToNullable(col);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
|
||||
block.insert(totals_without_keys.safeGetByPosition(i));
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_nam
|
||||
void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, const Block & block_right, const Names & key_names_right);
|
||||
|
||||
void createMissedColumns(Block & block);
|
||||
void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block);
|
||||
void joinTotals(const Block & totals, const Block & columns_to_add, const TableJoin & table_join, Block & block);
|
||||
|
||||
void addDefaultValues(IColumn & column, const DataTypePtr & type, size_t count);
|
||||
|
||||
|
@ -14,6 +14,7 @@ struct PullingAsyncPipelineExecutor::Data
|
||||
{
|
||||
PipelineExecutorPtr executor;
|
||||
std::exception_ptr exception;
|
||||
LazyOutputFormat * lazy_format = nullptr;
|
||||
std::atomic_bool is_finished = false;
|
||||
std::atomic_bool has_exception = false;
|
||||
ThreadFromGlobalPool thread;
|
||||
@ -82,6 +83,10 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou
|
||||
{
|
||||
data.exception = std::current_exception();
|
||||
data.has_exception = true;
|
||||
|
||||
/// Finish lazy format in case of exception. Otherwise thread.join() may hung.
|
||||
if (data.lazy_format)
|
||||
data.lazy_format->finalize();
|
||||
}
|
||||
|
||||
data.is_finished = true;
|
||||
@ -95,6 +100,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
{
|
||||
data = std::make_unique<Data>();
|
||||
data->executor = pipeline.execute();
|
||||
data->lazy_format = lazy_format.get();
|
||||
|
||||
auto func = [&, thread_group = CurrentThread::getGroup()]()
|
||||
{
|
||||
@ -105,14 +111,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
}
|
||||
|
||||
if (data->has_exception)
|
||||
{
|
||||
/// Finish lazy format in case of exception. Otherwise thread.join() may hung.
|
||||
if (lazy_format)
|
||||
lazy_format->finish();
|
||||
|
||||
data->has_exception = false;
|
||||
std::rethrow_exception(std::move(data->exception));
|
||||
}
|
||||
|
||||
bool is_execution_finished = lazy_format ? lazy_format->isFinished()
|
||||
: data->is_finished.load();
|
||||
@ -121,7 +120,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
{
|
||||
/// If lazy format is finished, we don't cancel pipeline but wait for main thread to be finished.
|
||||
data->is_finished = true;
|
||||
/// Wait thread ant rethrow exception if any.
|
||||
/// Wait thread and rethrow exception if any.
|
||||
cancel();
|
||||
return false;
|
||||
}
|
||||
@ -133,7 +132,12 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
}
|
||||
|
||||
chunk.clear();
|
||||
data->finish_event.tryWait(milliseconds);
|
||||
|
||||
if (milliseconds)
|
||||
data->finish_event.tryWait(milliseconds);
|
||||
else
|
||||
data->finish_event.wait();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -16,8 +16,13 @@ Chunk LazyOutputFormat::getChunk(UInt64 milliseconds)
|
||||
}
|
||||
|
||||
Chunk chunk;
|
||||
if (!queue.tryPop(chunk, milliseconds))
|
||||
return {};
|
||||
if (milliseconds)
|
||||
{
|
||||
if (!queue.tryPop(chunk, milliseconds))
|
||||
return {};
|
||||
}
|
||||
else
|
||||
queue.pop(chunk);
|
||||
|
||||
if (chunk)
|
||||
info.update(chunk.getNumRows(), chunk.allocatedBytes());
|
||||
|
@ -36,6 +36,14 @@ public:
|
||||
queue.clear();
|
||||
}
|
||||
|
||||
void finalize() override
|
||||
{
|
||||
finished_processing = true;
|
||||
|
||||
/// In case we are waiting for result.
|
||||
queue.emplace(Chunk());
|
||||
}
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override
|
||||
{
|
||||
@ -46,14 +54,6 @@ protected:
|
||||
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
|
||||
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
|
||||
|
||||
void finalize() override
|
||||
{
|
||||
finished_processing = true;
|
||||
|
||||
/// In case we are waiting for result.
|
||||
queue.emplace(Chunk());
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
ConcurrentBoundedQueue<Chunk> queue;
|
||||
|
@ -32,6 +32,8 @@ public:
|
||||
void describeActions(FormatSettings &) const override;
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
const Aggregator::Params & getParams() const { return params; }
|
||||
|
||||
private:
|
||||
Aggregator::Params params;
|
||||
bool final;
|
||||
|
@ -34,7 +34,7 @@ private:
|
||||
class CreatingSetsStep : public IQueryPlanStep
|
||||
{
|
||||
public:
|
||||
CreatingSetsStep(DataStreams input_streams_);
|
||||
explicit CreatingSetsStep(DataStreams input_streams_);
|
||||
|
||||
String getName() const override { return "CreatingSets"; }
|
||||
|
||||
|
@ -43,4 +43,9 @@ void CubeStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
const Aggregator::Params & CubeStep::getParams() const
|
||||
{
|
||||
return params->params;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -18,6 +19,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
const Aggregator::Params & getParams() const;
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
};
|
||||
|
@ -17,6 +17,8 @@ public:
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
const SortDescription & getSortDescription() const { return sort_description; }
|
||||
|
||||
private:
|
||||
SortDescription sort_description;
|
||||
};
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user