mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into ucasFL-union-distinct-improve
This commit is contained in:
commit
eb42896e26
@ -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:
|
||||
|
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
|
||||
|
@ -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.
|
||||
┏━━━━━━━━━━┳━━━━━━━━━━┓
|
||||
|
@ -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-->
|
||||
|
@ -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).
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
┏━━━━━━━━━━┳━━━━━━━━━━┓
|
||||
|
@ -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-->
|
||||
|
@ -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-->
|
||||
|
@ -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-->
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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,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>
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
|
@ -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);
|
||||
|
||||
|
@ -171,6 +171,8 @@ void WriteBufferFromHTTPServerResponse::finalize()
|
||||
try
|
||||
{
|
||||
next();
|
||||
if (out)
|
||||
out->finalize();
|
||||
out.reset();
|
||||
}
|
||||
catch (...)
|
||||
|
@ -13,7 +13,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/ConcatReadBuffer.h>
|
||||
#include <IO/HexWriteBuffer.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -147,7 +147,6 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
|
||||
}
|
||||
else if (action_type == ActionType::ADD_PART)
|
||||
{
|
||||
auto part_disk = storage.reserveSpace(0)->getDisk();
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
|
||||
|
||||
part = storage.createPart(
|
||||
|
@ -144,6 +144,12 @@ static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000;
|
||||
|
||||
void StorageReplicatedMergeTree::setZooKeeper()
|
||||
{
|
||||
/// Every ReplicatedMergeTree table is using only one ZooKeeper session.
|
||||
/// But if several ReplicatedMergeTree tables are using different
|
||||
/// ZooKeeper sessions, some queries like ATTACH PARTITION FROM may have
|
||||
/// strange effects. So we always use only one session for all tables.
|
||||
/// (excluding auxiliary zookeepers)
|
||||
|
||||
std::lock_guard lock(current_zookeeper_mutex);
|
||||
if (zookeeper_name == default_zookeeper_name)
|
||||
{
|
||||
|
@ -74,6 +74,9 @@ def test_reload_zookeeper(start_cluster):
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("SELECT COUNT() FROM test_table", settings={"select_sequential_consistency" : 1})
|
||||
|
||||
def get_active_zk_connections():
|
||||
return str(node.exec_in_container(['bash', '-c', 'lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l'], privileged=True, user='root')).strip()
|
||||
|
||||
## set config to zoo2, server will be normal
|
||||
new_config = """
|
||||
<yandex>
|
||||
@ -89,5 +92,10 @@ def test_reload_zookeeper(start_cluster):
|
||||
node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config)
|
||||
node.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
active_zk_connections = get_active_zk_connections()
|
||||
assert active_zk_connections == '1', "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)
|
||||
|
||||
assert_eq_with_retry(node, "SELECT COUNT() FROM test_table", '1000', retry_count=120, sleep_time=0.5)
|
||||
|
||||
active_zk_connections = get_active_zk_connections()
|
||||
assert active_zk_connections == '1', "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)
|
||||
|
@ -129,7 +129,7 @@ def test_secure_connection():
|
||||
# We need absolute path in zookeeper volumes. Generate it dynamically.
|
||||
TEMPLATE = '''
|
||||
zoo{zoo_id}:
|
||||
image: zookeeper:3.5.6
|
||||
image: zookeeper:3.6.2
|
||||
restart: always
|
||||
environment:
|
||||
ZOO_TICK_TIME: 500
|
||||
|
@ -11,3 +11,4 @@ ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip,deflate,br' "${CLICKHOUSE_URL}&
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip,deflate' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT 1' | gzip -d
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | gzip -d | tail -n3
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | brotli -d | tail -n3
|
||||
|
||||
|
@ -1,2 +1,6 @@
|
||||
|
||||
0
|
||||
|
||||
0
|
||||
|
||||
0 0 0
|
||||
|
@ -1,4 +1,6 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
CREATE TABLE t (`item_id` UInt64, `price_sold` Float32, `date` Date) ENGINE = MergeTree ORDER BY item_id;
|
||||
SELECT item_id FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id);
|
||||
SELECT item_id FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id) SETTINGS join_use_nulls = '1';
|
||||
SELECT * FROM (SELECT item_id, sum(price_sold) as price_sold FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id, sum(price_sold) as price_sold FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id) SETTINGS join_use_nulls = '1';
|
||||
DROP TABLE t;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
Here is CDTATA.
|
||||
This is a white space test.
|
||||
This is a complex test. <script type="text/javascript">Hello, world</script> world <style> hello
|
||||
This is a complex test.<script type="text/javascript">Hello, world</script> world <style> hello
|
||||
hello, world
|
||||
|
||||
hello, world
|
||||
|
@ -1,8 +1,9 @@
|
||||
SELECT htmlOrXmlCoarseParse('<script>Here is script.</script>');
|
||||
SELECT htmlOrXmlCoarseParse('<style>Here is style.</style>');
|
||||
SELECT htmlOrXmlCoarseParse('<![CDATA[Here is CDTATA.]]>');
|
||||
SELECT htmlOrXmlCoarseParse('This is a white space test.');
|
||||
SELECT htmlOrXmlCoarseParse('This is a complex test. <!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"\n "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd"><html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en"><![CDATA[<script type="text/javascript">Hello, world</script> ]]><hello />world<![CDATA[ <style> ]]> hello</style>\n<script><![CDATA[</script>]]>hello</script>\n</html>');
|
||||
SELECT extractTextFromHTML('<script>Here is script.</script>');
|
||||
SELECT extractTextFromHTML('<style>Here is style.</style>');
|
||||
SELECT extractTextFromHTML('<![CDATA[Here is CDTATA.]]>');
|
||||
SELECT extractTextFromHTML('This is a white space test.');
|
||||
SELECT extractTextFromHTML('This is a complex test. <!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"\n "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd"><html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en"><![CDATA[<script type="text/javascript">Hello, world</script> ]]><hello />world<![CDATA[ <style> ]]> hello</style>\n<script><![CDATA[</script>]]>hello</script>\n</html>');
|
||||
|
||||
DROP TABLE IF EXISTS defaults;
|
||||
CREATE TABLE defaults
|
||||
(
|
||||
@ -11,5 +12,5 @@ CREATE TABLE defaults
|
||||
|
||||
INSERT INTO defaults values ('<common tag>hello, world<tag>'), ('<script desc=content> some content </script>'), ('<![CDATA[hello, world]]>'), ('white space collapse');
|
||||
|
||||
SELECT htmlOrXmlCoarseParse(stringColumn) FROM defaults;
|
||||
SELECT extractTextFromHTML(stringColumn) FROM defaults;
|
||||
DROP table defaults;
|
||||
|
24
tests/queries/0_stateless/01698_fix_toMinute.reference
Normal file
24
tests/queries/0_stateless/01698_fix_toMinute.reference
Normal file
@ -0,0 +1,24 @@
|
||||
Check the bug causing situation: the special Australia/Lord_Howe time zone. toDateTime and toString functions are all tested at once
|
||||
1554559200 2019-04-07 01:00:00 2019-04-07 01:00:00
|
||||
1554559800 2019-04-07 01:10:00 2019-04-07 01:10:00
|
||||
1554560400 2019-04-07 01:20:00 2019-04-07 01:20:00
|
||||
1554561000 2019-04-07 01:30:00 2019-04-07 01:30:00
|
||||
1554561600 2019-04-07 01:40:00 2019-04-07 01:40:00
|
||||
1554562200 2019-04-07 01:50:00 2019-04-07 01:50:00
|
||||
1554562800 2019-04-07 01:30:00 2019-04-07 01:30:00
|
||||
1554563400 2019-04-07 01:40:00 2019-04-07 01:40:00
|
||||
1554564000 2019-04-07 01:50:00 2019-04-07 01:50:00
|
||||
1554564600 2019-04-07 02:00:00 2019-04-07 02:00:00
|
||||
1554565200 2019-04-07 02:10:00 2019-04-07 02:10:00
|
||||
1554565800 2019-04-07 02:20:00 2019-04-07 02:20:00
|
||||
1554566400 2019-04-07 02:30:00 2019-04-07 02:30:00
|
||||
1554567000 2019-04-07 02:40:00 2019-04-07 02:40:00
|
||||
1554567600 2019-04-07 02:50:00 2019-04-07 02:50:00
|
||||
1554568200 2019-04-07 03:00:00 2019-04-07 03:00:00
|
||||
1554568800 2019-04-07 03:10:00 2019-04-07 03:10:00
|
||||
1554569400 2019-04-07 03:20:00 2019-04-07 03:20:00
|
||||
1554570000 2019-04-07 03:30:00 2019-04-07 03:30:00
|
||||
1554570600 2019-04-07 03:40:00 2019-04-07 03:40:00
|
||||
4 days test in batch comparing with manually computation result for Europe/Moscow whose timezone epoc is of whole hour:
|
||||
4 days test in batch comparing with manually computation result for Asia/Tehran whose timezone epoc is of half hour:
|
||||
4 days test in batch comparing with manually computation result for Australia/Lord_Howe whose timezone epoc is of half hour and also its DST offset is half hour:
|
16
tests/queries/0_stateless/01698_fix_toMinute.sql
Normal file
16
tests/queries/0_stateless/01698_fix_toMinute.sql
Normal file
@ -0,0 +1,16 @@
|
||||
/* toDateTime or toString or other functions which should call the toMinute() function will all meet this bug. tests below will verify the toDateTime and toString. */
|
||||
SELECT 'Check the bug causing situation: the special Australia/Lord_Howe time zone. toDateTime and toString functions are all tested at once';
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, toString(x) as xx FROM numbers(20);
|
||||
|
||||
/* The Batch Part. Test period is whole 4 days*/
|
||||
SELECT '4 days test in batch comparing with manually computation result for Europe/Moscow whose timezone epoc is of whole hour:';
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('1981-04-01 00:00:00', 'Europe/Moscow') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('1981-09-30 00:00:00', 'Europe/Moscow') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
|
||||
SELECT '4 days test in batch comparing with manually computation result for Asia/Tehran whose timezone epoc is of half hour:';
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-03-21 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-09-20 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
|
||||
SELECT '4 days test in batch comparing with manually computation result for Australia/Lord_Howe whose timezone epoc is of half hour and also its DST offset is half hour:';
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-10-04 01:40:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
@ -50,57 +50,29 @@ DST boundary test for Australia/Lord_Howe. This is a special timezone with DST o
|
||||
DST boundary test for Australia/Lord_Howe:
|
||||
0 2020-10-04 01:40:00 37800 1601737800
|
||||
1 2020-10-04 01:50:00 37800 1601738400
|
||||
2 2020-10-04 02:00:00 39600 1601739000
|
||||
3 2020-10-04 02:10:00 39600 1601739600
|
||||
2 2020-10-04 02:30:00 39600 1601739000
|
||||
3 2020-10-04 02:40:00 39600 1601739600
|
||||
0 2019-04-07 01:00:00 39600 1554559200
|
||||
1 2019-04-07 01:10:00 39600 1554559800
|
||||
2 2019-04-07 01:20:00 39600 1554560400
|
||||
3 2019-04-07 01:30:00 39600 1554561000
|
||||
4 2019-04-07 01:40:00 39600 1554561600
|
||||
5 2019-04-07 01:50:00 39600 1554562200
|
||||
6 2019-04-07 01:00:00 37800 1554562800
|
||||
7 2019-04-07 01:10:00 37800 1554563400
|
||||
8 2019-04-07 01:20:00 37800 1554564000
|
||||
9 2019-04-07 02:30:00 37800 1554564600
|
||||
10 2019-04-07 02:40:00 37800 1554565200
|
||||
11 2019-04-07 02:50:00 37800 1554565800
|
||||
12 2019-04-07 02:00:00 37800 1554566400
|
||||
13 2019-04-07 02:10:00 37800 1554567000
|
||||
14 2019-04-07 02:20:00 37800 1554567600
|
||||
15 2019-04-07 03:30:00 37800 1554568200
|
||||
16 2019-04-07 03:40:00 37800 1554568800
|
||||
17 2019-04-07 03:50:00 37800 1554569400
|
||||
6 2019-04-07 01:30:00 37800 1554562800
|
||||
7 2019-04-07 01:40:00 37800 1554563400
|
||||
8 2019-04-07 01:50:00 37800 1554564000
|
||||
9 2019-04-07 02:00:00 37800 1554564600
|
||||
10 2019-04-07 02:10:00 37800 1554565200
|
||||
11 2019-04-07 02:20:00 37800 1554565800
|
||||
12 2019-04-07 02:30:00 37800 1554566400
|
||||
13 2019-04-07 02:40:00 37800 1554567000
|
||||
14 2019-04-07 02:50:00 37800 1554567600
|
||||
15 2019-04-07 03:00:00 37800 1554568200
|
||||
16 2019-04-07 03:10:00 37800 1554568800
|
||||
17 2019-04-07 03:20:00 37800 1554569400
|
||||
4 days test in batch comparing with manually computation result for Europe/Moscow:
|
||||
4 days test in batch comparing with manually computation result for Asia/Tehran:
|
||||
The result maybe wrong for toDateTime processing Australia/Lord_Howe
|
||||
1601739000 2020-10-04 02:00:00 39600 37800
|
||||
1601739600 2020-10-04 02:10:00 39600 37800
|
||||
1601740200 2020-10-04 02:20:00 39600 37800
|
||||
1601740800 2020-10-04 03:30:00 39600 41400
|
||||
1601741400 2020-10-04 03:40:00 39600 41400
|
||||
1601742000 2020-10-04 03:50:00 39600 41400
|
||||
1601742600 2020-10-04 03:00:00 39600 37800
|
||||
1601743200 2020-10-04 03:10:00 39600 37800
|
||||
1601743800 2020-10-04 03:20:00 39600 37800
|
||||
1601744400 2020-10-04 04:30:00 39600 41400
|
||||
1601745000 2020-10-04 04:40:00 39600 41400
|
||||
1601745600 2020-10-04 04:50:00 39600 41400
|
||||
1601746200 2020-10-04 04:00:00 39600 37800
|
||||
1601746800 2020-10-04 04:10:00 39600 37800
|
||||
1601747400 2020-10-04 04:20:00 39600 37800
|
||||
1601748000 2020-10-04 05:30:00 39600 41400
|
||||
1554562800 2019-04-07 01:00:00 37800 36000
|
||||
1554563400 2019-04-07 01:10:00 37800 36000
|
||||
1554564000 2019-04-07 01:20:00 37800 36000
|
||||
1554564600 2019-04-07 02:30:00 37800 39600
|
||||
1554565200 2019-04-07 02:40:00 37800 39600
|
||||
1554565800 2019-04-07 02:50:00 37800 39600
|
||||
1554566400 2019-04-07 02:00:00 37800 36000
|
||||
1554567000 2019-04-07 02:10:00 37800 36000
|
||||
1554567600 2019-04-07 02:20:00 37800 36000
|
||||
1554568200 2019-04-07 03:30:00 37800 39600
|
||||
1554568800 2019-04-07 03:40:00 37800 39600
|
||||
1554569400 2019-04-07 03:50:00 37800 39600
|
||||
4 days test in batch comparing with manually computation result for Australia/Lord_Howe
|
||||
Moscow DST Years:
|
||||
11 1981-06-01 00:00:00 14400
|
||||
12 1982-06-01 00:00:00 14400
|
||||
|
@ -26,8 +26,7 @@ SELECT '4 days test in batch comparing with manually computation result for Asia
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-03-21 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-09-20 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc;
|
||||
|
||||
/* During this test we got unexpected result comes from the toDateTime() function when process the special time zone of 'Australia/Lord_Howe', which may be some kind of bugs. */
|
||||
SELECT 'The result maybe wrong for toDateTime processing Australia/Lord_Howe';
|
||||
SELECT '4 days test in batch comparing with manually computation result for Australia/Lord_Howe';
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-10-04 01:40:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(18) where res != calc;
|
||||
SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(18) where res != calc;
|
||||
|
||||
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
2
|
||||
3
|
@ -0,0 +1,47 @@
|
||||
SET allow_suspicious_low_cardinality_types = 1;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_on_nullable_type;
|
||||
CREATE TABLE constraint_on_nullable_type
|
||||
(
|
||||
`id` Nullable(UInt64),
|
||||
CONSTRAINT `c0` CHECK `id` = 1
|
||||
)
|
||||
ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO constraint_on_nullable_type VALUES (0); -- {serverError 469}
|
||||
INSERT INTO constraint_on_nullable_type VALUES (1);
|
||||
|
||||
SELECT * FROM constraint_on_nullable_type;
|
||||
|
||||
DROP TABLE constraint_on_nullable_type;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_on_low_cardinality_type;
|
||||
CREATE TABLE constraint_on_low_cardinality_type
|
||||
(
|
||||
`id` LowCardinality(UInt64),
|
||||
CONSTRAINT `c0` CHECK `id` = 2
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_on_low_cardinality_type VALUES (0); -- {serverError 469}
|
||||
INSERT INTO constraint_on_low_cardinality_type VALUES (2);
|
||||
|
||||
SELECT * FROM constraint_on_low_cardinality_type;
|
||||
|
||||
DROP TABLE constraint_on_low_cardinality_type;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_on_low_cardinality_nullable_type;
|
||||
|
||||
CREATE TABLE constraint_on_low_cardinality_nullable_type
|
||||
(
|
||||
`id` LowCardinality(Nullable(UInt64)),
|
||||
CONSTRAINT `c0` CHECK `id` = 3
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (0); -- {serverError 469}
|
||||
INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (3);
|
||||
|
||||
SELECT * FROM constraint_on_low_cardinality_nullable_type;
|
||||
|
||||
DROP TABLE constraint_on_low_cardinality_nullable_type;
|
@ -0,0 +1 @@
|
||||
1
|
@ -0,0 +1,40 @@
|
||||
DROP TABLE IF EXISTS constraint_constant_number_expression;
|
||||
CREATE TABLE constraint_constant_number_expression
|
||||
(
|
||||
id UInt64,
|
||||
CONSTRAINT `c0` CHECK 1,
|
||||
CONSTRAINT `c1` CHECK 1 < 2,
|
||||
CONSTRAINT `c2` CHECK isNull(cast(NULL, 'Nullable(UInt8)'))
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO constraint_constant_number_expression VALUES (1);
|
||||
|
||||
SELECT * FROM constraint_constant_number_expression;
|
||||
|
||||
DROP TABLE constraint_constant_number_expression;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_constant_number_expression_non_uint8;
|
||||
CREATE TABLE constraint_constant_number_expression_non_uint8
|
||||
(
|
||||
id UInt64,
|
||||
CONSTRAINT `c0` CHECK toUInt64(1)
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO constraint_constant_number_expression_non_uint8 VALUES (2); -- {serverError 1}
|
||||
|
||||
SELECT * FROM constraint_constant_number_expression_non_uint8;
|
||||
|
||||
DROP TABLE constraint_constant_number_expression_non_uint8;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_constant_nullable_expression_that_contains_null;
|
||||
CREATE TABLE constraint_constant_nullable_expression_that_contains_null
|
||||
(
|
||||
id UInt64,
|
||||
CONSTRAINT `c0` CHECK nullIf(1 % 2, 1)
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO constraint_constant_nullable_expression_that_contains_null VALUES (3); -- {serverError 469}
|
||||
|
||||
SELECT * FROM constraint_constant_nullable_expression_that_contains_null;
|
||||
|
||||
DROP TABLE constraint_constant_nullable_expression_that_contains_null;
|
@ -0,0 +1,23 @@
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999998"
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999999"
|
||||
}
|
||||
],
|
||||
|
||||
"rows": 1000000,
|
||||
|
||||
"rows_before_limit_at_least": 1048080,
|
||||
|
@ -0,0 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23
|
120
tests/queries/0_stateless/01746_extract_text_from_html.reference
Normal file
120
tests/queries/0_stateless/01746_extract_text_from_html.reference
Normal file
@ -0,0 +1,120 @@
|
||||
-- { echo }
|
||||
|
||||
SELECT extractTextFromHTML('');
|
||||
|
||||
SELECT extractTextFromHTML(' ');
|
||||
|
||||
SELECT extractTextFromHTML(' ');
|
||||
|
||||
SELECT extractTextFromHTML('Hello');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello, world');
|
||||
Hello, world
|
||||
SELECT extractTextFromHTML('Hello, world');
|
||||
Hello, world
|
||||
SELECT extractTextFromHTML(' Hello, world');
|
||||
Hello, world
|
||||
SELECT extractTextFromHTML(' Hello, world ');
|
||||
Hello, world
|
||||
SELECT extractTextFromHTML(' \t Hello,\rworld \n ');
|
||||
Hello, world
|
||||
SELECT extractTextFromHTML('Hello<world');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello < world');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello > world');
|
||||
Hello > world
|
||||
SELECT extractTextFromHTML('Hello<world>');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello<>world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello<!>world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello<!->world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello<!-->world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello<!--->world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello<!---->world');
|
||||
Hello world
|
||||
SELECT extractTextFromHTML('Hello <!-- --> World');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello<!-- --> World');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello<!-- -->World');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello <!-- -->World');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello <u> World</u>');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello <u>World</u>');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello<u>World</u>');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('Hello<u> World</u>');
|
||||
Hello World
|
||||
SELECT extractTextFromHTML('<![CDATA[ \t Hello,\rworld \n ]]>');
|
||||
\t Hello,\rworld \n
|
||||
SELECT extractTextFromHTML('Hello <![CDATA[Hello\tworld]]> world!');
|
||||
HelloHello\tworld world!
|
||||
SELECT extractTextFromHTML('Hello<![CDATA[Hello\tworld]]>world!');
|
||||
HelloHello\tworld world!
|
||||
SELECT extractTextFromHTML('Hello <![CDATA[Hello <b>world</b>]]> world!');
|
||||
HelloHello <b>world</b> world!
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John Smith</sender>]]>');
|
||||
<sender>John Smith</sender>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John <![CDATA[Smith</sender>]]>');
|
||||
<sender>John <![CDATA[Smith</sender>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John <![CDATA[]]>Smith</sender>]]>');
|
||||
<sender>John <![CDATA[ Smith ]]>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]><![CDATA[Smith</sender>]]>');
|
||||
<sender>John Smith</sender>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]> <![CDATA[Smith</sender>]]>');
|
||||
<sender>John Smith</sender>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John]]> <![CDATA[Smith</sender>]]>');
|
||||
<sender>JohnSmith</sender>
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]>]]><![CDATA[Smith</sender>]]>');
|
||||
<sender>John ]]>Smith</sender>
|
||||
SELECT extractTextFromHTML('Hello<script>World</script> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<script >World</script> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<scripta>World</scripta> goodbye');
|
||||
Hello World goodbye
|
||||
SELECT extractTextFromHTML('Hello<script type="text/javascript">World</script> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World</style> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<script:p>World</script:p> goodbye');
|
||||
Hello World goodbye
|
||||
SELECT extractTextFromHTML('Hello<script:p type="text/javascript">World</script:p> goodbye');
|
||||
Hello World goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </style> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </style \n > goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </ style> goodbye');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </stylea> goodbye');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDATA[</style>]]> </stylea> goodbye');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDATA[</style>]]> </style> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDAT[</style>]]> </style> goodbye');
|
||||
Hello ]]> goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![endif]--> </style> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <script>abc</script> </stylea> goodbye');
|
||||
Hello
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <script>abc</script> </style> goodbye');
|
||||
Hello goodbye
|
||||
SELECT extractTextFromHTML('<![CDATA[]]]]><![CDATA[>]]>');
|
||||
]]>
|
||||
SELECT extractTextFromHTML('
|
||||
<img src="pictures/power.png" style="margin-bottom: -30px;" />
|
||||
<br><span style="padding-right: 10px; font-size: 10px;">xkcd.com</span>
|
||||
</div>
|
||||
');
|
||||
xkcd.com
|
72
tests/queries/0_stateless/01746_extract_text_from_html.sql
Normal file
72
tests/queries/0_stateless/01746_extract_text_from_html.sql
Normal file
@ -0,0 +1,72 @@
|
||||
-- { echo }
|
||||
|
||||
SELECT extractTextFromHTML('');
|
||||
SELECT extractTextFromHTML(' ');
|
||||
SELECT extractTextFromHTML(' ');
|
||||
SELECT extractTextFromHTML('Hello');
|
||||
SELECT extractTextFromHTML('Hello, world');
|
||||
SELECT extractTextFromHTML('Hello, world');
|
||||
SELECT extractTextFromHTML(' Hello, world');
|
||||
SELECT extractTextFromHTML(' Hello, world ');
|
||||
SELECT extractTextFromHTML(' \t Hello,\rworld \n ');
|
||||
|
||||
SELECT extractTextFromHTML('Hello<world');
|
||||
SELECT extractTextFromHTML('Hello < world');
|
||||
SELECT extractTextFromHTML('Hello > world');
|
||||
SELECT extractTextFromHTML('Hello<world>');
|
||||
SELECT extractTextFromHTML('Hello<>world');
|
||||
SELECT extractTextFromHTML('Hello<!>world');
|
||||
SELECT extractTextFromHTML('Hello<!->world');
|
||||
SELECT extractTextFromHTML('Hello<!-->world');
|
||||
SELECT extractTextFromHTML('Hello<!--->world');
|
||||
SELECT extractTextFromHTML('Hello<!---->world');
|
||||
|
||||
SELECT extractTextFromHTML('Hello <!-- --> World');
|
||||
SELECT extractTextFromHTML('Hello<!-- --> World');
|
||||
SELECT extractTextFromHTML('Hello<!-- -->World');
|
||||
SELECT extractTextFromHTML('Hello <!-- -->World');
|
||||
SELECT extractTextFromHTML('Hello <u> World</u>');
|
||||
SELECT extractTextFromHTML('Hello <u>World</u>');
|
||||
SELECT extractTextFromHTML('Hello<u>World</u>');
|
||||
SELECT extractTextFromHTML('Hello<u> World</u>');
|
||||
|
||||
SELECT extractTextFromHTML('<![CDATA[ \t Hello,\rworld \n ]]>');
|
||||
SELECT extractTextFromHTML('Hello <![CDATA[Hello\tworld]]> world!');
|
||||
SELECT extractTextFromHTML('Hello<![CDATA[Hello\tworld]]>world!');
|
||||
|
||||
SELECT extractTextFromHTML('Hello <![CDATA[Hello <b>world</b>]]> world!');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John <![CDATA[Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John <![CDATA[]]>Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]><![CDATA[Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]> <![CDATA[Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John]]> <![CDATA[Smith</sender>]]>');
|
||||
SELECT extractTextFromHTML('<![CDATA[<sender>John ]]>]]><![CDATA[Smith</sender>]]>');
|
||||
|
||||
SELECT extractTextFromHTML('Hello<script>World</script> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<script >World</script> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<scripta>World</scripta> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<script type="text/javascript">World</script> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World</style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<script:p>World</script:p> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<script:p type="text/javascript">World</script:p> goodbye');
|
||||
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </style \n > goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </ style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <!-- abc --> </stylea> goodbye');
|
||||
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDATA[</style>]]> </stylea> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDATA[</style>]]> </style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![CDAT[</style>]]> </style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <![endif]--> </style> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <script>abc</script> </stylea> goodbye');
|
||||
SELECT extractTextFromHTML('Hello<style type="text/css">World <script>abc</script> </style> goodbye');
|
||||
|
||||
SELECT extractTextFromHTML('<![CDATA[]]]]><![CDATA[>]]>');
|
||||
|
||||
SELECT extractTextFromHTML('
|
||||
<img src="pictures/power.png" style="margin-bottom: -30px;" />
|
||||
<br><span style="padding-right: 10px; font-size: 10px;">xkcd.com</span>
|
||||
</div>
|
||||
');
|
@ -0,0 +1,23 @@
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999998"
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999999"
|
||||
}
|
||||
],
|
||||
|
||||
"rows": 1000000,
|
||||
|
||||
"rows_before_limit_at_least": 1048080,
|
||||
|
@ -0,0 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
|
@ -0,0 +1,23 @@
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999998"
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
"pipeline": "test-pipeline",
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999999"
|
||||
}
|
||||
],
|
||||
|
||||
"rows": 1000000,
|
||||
|
||||
"rows_before_limit_at_least": 1048080,
|
||||
|
@ -0,0 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | zstd -d | tail -n30 | head -n23
|
@ -260,7 +260,8 @@
|
||||
"00121_drop_column_zookeeper",
|
||||
"00116_storage_set",
|
||||
"00083_create_merge_tree_zookeeper",
|
||||
"00062_replicated_merge_tree_alter_zookeeper"
|
||||
"00062_replicated_merge_tree_alter_zookeeper",
|
||||
"01720_constraints_complex_types"
|
||||
],
|
||||
"polymorphic-parts": [
|
||||
"01508_partition_pruning_long", /// bug, shoud be fixed
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -2,7 +2,7 @@ version: '2.3'
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
image: zookeeper:3.4.12
|
||||
image: zookeeper:3.6.2
|
||||
expose:
|
||||
- "2181"
|
||||
environment:
|
||||
|
@ -11,7 +11,7 @@
|
||||
.syntax .hll { background-color: #b9b6b0 }
|
||||
.syntax { background: #f8f9fa; color: #2f1e2e }
|
||||
.syntax .c { color: #8d8687 } /* Comment */
|
||||
.syntax .err { color: #ef6155 } /* Error */
|
||||
.syntax .err {} /* Error */
|
||||
.syntax .k { color: #000000; font-weight: bold } /* Keyword */
|
||||
.syntax .l { color: #0088ff } /* Literal */
|
||||
.syntax .n { color: #2f1e2e } /* Name */
|
||||
|
Loading…
Reference in New Issue
Block a user