Merge branch 'master' into ucasFL-union-distinct-improve

This commit is contained in:
Nikolai Kochetov 2021-03-01 10:43:39 +03:00
commit eb42896e26
98 changed files with 1828 additions and 1196 deletions

View File

@ -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; }

View File

@ -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
{

View File

@ -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();
}
}

View File

@ -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

View File

@ -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

@ -1 +1 @@
Subproject commit 5805f99a533a8f8118699c0100d8c102f3605f65
Subproject commit 63be8a99401992075c23e99f7c84de1c653e39e2

View File

@ -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})

View File

@ -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"

View File

@ -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 \

View File

@ -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:

File diff suppressed because one or more lines are too long

View File

@ -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-->

View File

@ -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

View File

@ -91,6 +91,8 @@ $ clickhouse-local --query "
Now lets 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.
┏━━━━━━━━━━┳━━━━━━━━━━┓

View File

@ -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-->

View File

@ -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-->

View File

@ -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.

View File

@ -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-->

View File

@ -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).

View File

@ -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)

View File

@ -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.
┏━━━━━━━━━━┳━━━━━━━━━━┓

View File

@ -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 dont 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-->

View File

@ -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-->

View File

@ -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-->

View File

@ -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

View File

@ -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;
};
}

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -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

View File

@ -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;
};
}

View 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>();
}
}

View File

@ -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

View File

@ -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);

View File

@ -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

View File

@ -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();
}
};
}

View File

@ -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;
}

View File

@ -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;

View File

@ -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();
}
}

View File

@ -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;
};
}

View File

@ -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;
}

View File

@ -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;

View File

@ -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;
}

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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;
};
}

View File

@ -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)

View File

@ -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;
}

View File

@ -28,7 +28,7 @@ try
DB::writeIntText(i, lzma_buf);
DB::writeChar('\t', lzma_buf);
}
lzma_buf.finish();
lzma_buf.finalize();
stopwatch.stop();

View File

@ -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();

View File

@ -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."

View File

@ -30,7 +30,7 @@ try
DB::writeIntText(i, zstd_buf);
DB::writeChar('\t', zstd_buf);
}
zstd_buf.finish();
zstd_buf.finalize();
stopwatch.stop();

View File

@ -29,7 +29,6 @@ SRCS(
HTTPChunkedReadBuffer.cpp
HTTPCommon.cpp
HashingWriteBuffer.cpp
HexWriteBuffer.cpp
LZMADeflatingWriteBuffer.cpp
LZMAInflatingReadBuffer.cpp
LimitReadBuffer.cpp

View File

@ -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>

View File

@ -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

View File

@ -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);
}

View File

@ -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;

View File

@ -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()

View File

@ -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));
}

View File

@ -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);

View File

@ -171,6 +171,8 @@ void WriteBufferFromHTTPServerResponse::finalize()
try
{
next();
if (out)
out->finalize();
out.reset();
}
catch (...)

View File

@ -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>

View File

@ -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(

View File

@ -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)
{

View File

@ -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)

View File

@ -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

View File

@ -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

View File

@ -1,2 +1,6 @@
0
0
0 0 0

View File

@ -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;

View File

@ -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

View File

@ -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;

View 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:

View 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;

View File

@ -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

View File

@ -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;

View File

@ -0,0 +1,3 @@
1
2
3

View File

@ -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;

View File

@ -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;

View File

@ -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,

View File

@ -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

View 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

View 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>
');

View File

@ -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,

View File

@ -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

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -2,7 +2,7 @@ version: '2.3'
services:
zookeeper:
image: zookeeper:3.4.12
image: zookeeper:3.6.2
expose:
- "2181"
environment:

View File

@ -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 */