mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'ClickHouse:master' into fix/signal_pipe_buf_size
This commit is contained in:
commit
64ae09b92d
2
contrib/lz4
vendored
2
contrib/lz4
vendored
@ -1 +1 @@
|
||||
Subproject commit f39b79fb02962a1cd880bbdecb6dffba4f754a11
|
||||
Subproject commit 4c9431e9af596af0556e5da0ae99305bafb2b10b
|
@ -159,8 +159,7 @@ Configuration fields:
|
||||
| Tag | Description | Required |
|
||||
|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|
|
||||
| `name` | Column name. | Yes |
|
||||
| `type` | ClickHouse data type: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md),
|
||||
[Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md), [String](../../../sql-reference/data-types/string.md), [Array](../../../sql-reference/data-types/array.md).<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
|
||||
| `type` | ClickHouse data type: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md),[Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md), [String](../../../sql-reference/data-types/string.md), [Array](../../../sql-reference/data-types/array.md).<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
|
||||
| `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. [NULL](../../syntax.md#null-literal) value can be used only for the `Nullable` types (see the previous line with types description). | Yes |
|
||||
| `expression` | [Expression](../../../sql-reference/syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No |
|
||||
| <a name="hierarchical-dict-attr"></a> `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md).<br/><br/>Default value: `false`. | No |
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -26,7 +26,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer(
|
||||
0 /* no dictID */,
|
||||
LZ4F_noBlockChecksum},
|
||||
compression_level, /* compression level; 0 == default */
|
||||
0, /* autoflush */
|
||||
1, /* autoflush */
|
||||
0, /* favor decompression speed */
|
||||
{0, 0, 0}, /* reserved, must be set to 0 */
|
||||
};
|
||||
@ -125,6 +125,8 @@ void Lz4DeflatingWriteBuffer::nextImpl()
|
||||
out->position() = out->buffer().begin();
|
||||
throw;
|
||||
}
|
||||
out->next();
|
||||
out_capacity = out->buffer().end() - out->position();
|
||||
}
|
||||
|
||||
void Lz4DeflatingWriteBuffer::finalizeBefore()
|
||||
|
@ -70,6 +70,12 @@ bool Lz4InflatingReadBuffer::nextImpl()
|
||||
return !working_buffer.empty();
|
||||
}
|
||||
|
||||
/// It may happen that we didn't get new uncompressed data
|
||||
/// (for example if we read the end of frame). Load new data
|
||||
/// in this case.
|
||||
if (working_buffer.empty())
|
||||
return nextImpl();
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -63,7 +63,10 @@ public:
|
||||
if (!res)
|
||||
working_buffer = Buffer(pos, pos);
|
||||
else
|
||||
{
|
||||
pos = working_buffer.begin() + nextimpl_working_buffer_offset;
|
||||
assert(position() != working_buffer.end());
|
||||
}
|
||||
nextimpl_working_buffer_offset = 0;
|
||||
|
||||
assert(position() <= working_buffer.end());
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <IO/readFloatText.h>
|
||||
|
@ -38,9 +38,22 @@ class DockerImage:
|
||||
self.parent = parent
|
||||
self.built = False
|
||||
|
||||
def __eq__(self, other):
|
||||
def __eq__(self, other) -> bool: # type: ignore
|
||||
"""Is used to check if DockerImage is in a set or not"""
|
||||
return self.path == other.path
|
||||
return self.path == other.path and self.repo == self.repo
|
||||
|
||||
def __lt__(self, other) -> bool:
|
||||
if not isinstance(other, DockerImage):
|
||||
return False
|
||||
if self.parent and not other.parent:
|
||||
return False
|
||||
if not self.parent and other.parent:
|
||||
return True
|
||||
if self.path < other.path:
|
||||
return True
|
||||
if self.repo < other.repo:
|
||||
return True
|
||||
return False
|
||||
|
||||
def __hash__(self):
|
||||
return hash(self.path)
|
||||
@ -49,7 +62,7 @@ class DockerImage:
|
||||
return self.repo
|
||||
|
||||
def __repr__(self):
|
||||
return f"DockerImage(path={self.path},path={self.path},parent={self.parent})"
|
||||
return f"DockerImage(path={self.path},repo={self.repo},parent={self.parent})"
|
||||
|
||||
|
||||
def get_changed_docker_images(
|
||||
@ -105,7 +118,9 @@ def get_changed_docker_images(
|
||||
dependent,
|
||||
image,
|
||||
)
|
||||
changed_images.append(DockerImage(dependent, image.repo, image))
|
||||
changed_images.append(
|
||||
DockerImage(dependent, images_dict[dependent]["name"], image)
|
||||
)
|
||||
index += 1
|
||||
if index > 5 * len(images_dict):
|
||||
# Sanity check to prevent infinite loop.
|
||||
|
@ -22,24 +22,59 @@ class TestDockerImageCheck(unittest.TestCase):
|
||||
"docker/test/base",
|
||||
"docker/docs/builder",
|
||||
}
|
||||
images = di.get_changed_docker_images(pr_info, "/", self.docker_images_path)
|
||||
expected = {
|
||||
di.DockerImage("docker/test/base", "clickhouse/test-base"),
|
||||
di.DockerImage("docker/docs/builder", "clickhouse/docs-builder"),
|
||||
di.DockerImage("docker/test/stateless", "clickhouse/stateless-test"),
|
||||
di.DockerImage(
|
||||
"docker/test/integration/base", "clickhouse/integration-test"
|
||||
),
|
||||
di.DockerImage("docker/test/fuzzer", "clickhouse/fuzzer"),
|
||||
di.DockerImage(
|
||||
"docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test"
|
||||
),
|
||||
di.DockerImage("docker/docs/check", "clickhouse/docs-check"),
|
||||
di.DockerImage("docker/docs/release", "clickhouse/docs-release"),
|
||||
di.DockerImage("docker/test/stateful", "clickhouse/stateful-test"),
|
||||
di.DockerImage("docker/test/unit", "clickhouse/unit-test"),
|
||||
di.DockerImage("docker/test/stress", "clickhouse/stress-test"),
|
||||
}
|
||||
images = sorted(
|
||||
list(di.get_changed_docker_images(pr_info, "/", self.docker_images_path))
|
||||
)
|
||||
self.maxDiff = None
|
||||
expected = sorted(
|
||||
[
|
||||
di.DockerImage("docker/test/base", "clickhouse/test-base"),
|
||||
di.DockerImage("docker/docs/builder", "clickhouse/docs-builder"),
|
||||
di.DockerImage(
|
||||
"docker/test/stateless",
|
||||
"clickhouse/stateless-test",
|
||||
"clickhouse/test-base",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/integration/base",
|
||||
"clickhouse/integration-test",
|
||||
"clickhouse/test-base",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/fuzzer", "clickhouse/fuzzer", "clickhouse/test-base"
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/keeper-jepsen",
|
||||
"clickhouse/keeper-jepsen-test",
|
||||
"clickhouse/test-base",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/docs/check",
|
||||
"clickhouse/docs-check",
|
||||
"clickhouse/docs-builder",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/docs/release",
|
||||
"clickhouse/docs-release",
|
||||
"clickhouse/docs-builder",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/stateful",
|
||||
"clickhouse/stateful-test",
|
||||
"clickhouse/stateless-test",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/unit",
|
||||
"clickhouse/unit-test",
|
||||
"clickhouse/stateless-test",
|
||||
),
|
||||
di.DockerImage(
|
||||
"docker/test/stress",
|
||||
"clickhouse/stress-test",
|
||||
"clickhouse/stateful-test",
|
||||
),
|
||||
]
|
||||
)
|
||||
self.assertEqual(images, expected)
|
||||
|
||||
def test_gen_version(self):
|
||||
|
@ -1,11 +1,14 @@
|
||||
-- Tags: no-parallel
|
||||
-- ^^^^^^^^^^^ otherwise you may hit TOO_DEEP_RECURSION error during querying system.columns
|
||||
|
||||
DROP TABLE IF EXISTS merge1;
|
||||
DROP TABLE IF EXISTS merge2;
|
||||
|
||||
CREATE TABLE IF NOT EXISTS merge1 (x UInt64) ENGINE = Merge(currentDatabase(), '^merge\\d$');
|
||||
CREATE TABLE IF NOT EXISTS merge2 (x UInt64) ENGINE = Merge(currentDatabase(), '^merge\\d$');
|
||||
|
||||
SELECT * FROM merge1; -- { serverError 306 }
|
||||
SELECT * FROM merge2; -- { serverError 306 }
|
||||
SELECT * FROM merge1; -- { serverError TOO_DEEP_RECURSION }
|
||||
SELECT * FROM merge2; -- { serverError TOO_DEEP_RECURSION }
|
||||
|
||||
DROP TABLE merge1;
|
||||
DROP TABLE merge2;
|
||||
|
@ -12,7 +12,6 @@ do
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE file (x UInt64) ENGINE = File(TSV, '${CLICKHOUSE_DATABASE}/${m}.tsv.${m}')"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE file"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO file SELECT * FROM numbers(1000000)"
|
||||
sleep 1
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x) FROM file"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE file"
|
||||
done
|
||||
|
@ -0,0 +1,45 @@
|
||||
Native
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
Values
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
JSONCompactEachRow
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
TSKV
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
TSV
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
CSV
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
JSONEachRow
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
JSONCompactEachRow
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
||||
JSONStringsEachRow
|
||||
9999
|
||||
99999
|
||||
999999
|
||||
2499999
|
21
tests/queries/0_stateless/02125_lz4_compression_bug.sh
Executable file
21
tests/queries/0_stateless/02125_lz4_compression_bug.sh
Executable file
@ -0,0 +1,21 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
for format in Native Values JSONCompactEachRow TSKV TSV CSV JSONEachRow JSONCompactEachRow JSONStringsEachRow
|
||||
do
|
||||
echo $format
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS file"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE file (x UInt64) ENGINE = File($format, '${CLICKHOUSE_DATABASE}/data.$format.lz4')"
|
||||
for size in 10000 100000 1000000 2500000
|
||||
do
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE file"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO file SELECT * FROM numbers($size)"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT max(x) FROM file"
|
||||
done
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE file"
|
@ -184,7 +184,9 @@ tables_with_database_column=(
|
||||
tests_with_database_column=( $(
|
||||
find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") | cut -d: -f1 | sort -u
|
||||
xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") |
|
||||
grep -v -e ':--' -e ':#' |
|
||||
cut -d: -f1 | sort -u
|
||||
) )
|
||||
for test_case in "${tests_with_database_column[@]}"; do
|
||||
grep -qE database.*currentDatabase "$test_case" || {
|
||||
|
Loading…
Reference in New Issue
Block a user