mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into fix_integration_tests_parser_bug
This commit is contained in:
commit
8438c17685
@ -83,15 +83,15 @@ def make_query_command(query):
|
||||
def prepare_for_hung_check(drop_databases):
|
||||
# FIXME this function should not exist, but...
|
||||
|
||||
# ThreadFuzzer significantly slows down server and causes false-positive hung check failures
|
||||
call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'")
|
||||
|
||||
# We attach gdb to clickhouse-server before running tests
|
||||
# to print stacktraces of all crashes even if clickhouse cannot print it for some reason.
|
||||
# However, it obstruct checking for hung queries.
|
||||
logging.info("Will terminate gdb (if any)")
|
||||
call_with_retry("kill -TERM $(pidof gdb)")
|
||||
|
||||
# ThreadFuzzer significantly slows down server and causes false-positive hung check failures
|
||||
call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'")
|
||||
|
||||
call_with_retry(make_query_command('SELECT 1 FORMAT Null'))
|
||||
|
||||
# Some tests execute SYSTEM STOP MERGES or similar queries.
|
||||
|
@ -11,10 +11,6 @@ To work with data stored on `Amazon S3` disks use [S3](../engines/table-engines/
|
||||
|
||||
To load data from a web server with static files use a disk with type [web](#storing-data-on-webserver).
|
||||
|
||||
## Zero-copy Replication {#zero-copy}
|
||||
|
||||
ClickHouse supports zero-copy replication for `S3` and `HDFS` disks, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself.
|
||||
|
||||
## Configuring HDFS {#configuring-hdfs}
|
||||
|
||||
[MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`.
|
||||
@ -316,3 +312,8 @@ When loading files by `endpoint`, they must be loaded into `<endpoint>/store/` p
|
||||
If URL is not reachable on disk load when the server is starting up tables, then all errors are caught. If in this case there were errors, tables can be reloaded (become visible) via `DETACH TABLE table_name` -> `ATTACH TABLE table_name`. If metadata was successfully loaded at server startup, then tables are available straight away.
|
||||
|
||||
Use [http_max_single_read_retries](../operations/settings/settings.md#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read.
|
||||
|
||||
|
||||
## Zero-copy Replication (not ready for production) {#zero-copy}
|
||||
|
||||
ClickHouse supports zero-copy replication for `S3` and `HDFS` disks, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself.
|
||||
|
@ -62,7 +62,7 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t
|
||||
{
|
||||
/// Nothing to do if we already at required position
|
||||
if (!size_compressed && static_cast<size_t>(file_in.getPosition()) == offset_in_compressed_file && /// correct position in compressed file
|
||||
(offset() == offset_in_decompressed_block /// correct position in buffer or
|
||||
((!buffer().empty() && offset() == offset_in_decompressed_block) /// correct position in buffer or
|
||||
|| nextimpl_working_buffer_offset == offset_in_decompressed_block)) /// we will move our position to correct one
|
||||
return;
|
||||
|
||||
|
@ -191,6 +191,33 @@ static void checkASTStructure(const ASTPtr & child)
|
||||
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
||||
static void autoAssignNumberForEnum(const ASTPtr & arguments)
|
||||
{
|
||||
UInt64 literal_child_count = 0;
|
||||
UInt64 func_child_count = 0;
|
||||
ASTs assign_number_child;
|
||||
assign_number_child.reserve(arguments->children.size());
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
{
|
||||
if (child->as<ASTLiteral>())
|
||||
{
|
||||
ASTPtr func = makeASTFunction("equals", child, std::make_shared<ASTLiteral>(++literal_child_count));
|
||||
assign_number_child.emplace_back(func);
|
||||
}
|
||||
else
|
||||
{
|
||||
++func_child_count;
|
||||
assign_number_child.emplace_back(child);
|
||||
}
|
||||
}
|
||||
|
||||
if (func_child_count > 0 && literal_child_count > 0)
|
||||
throw Exception("ALL Elements of Enum data type must be of form: 'name' = number or 'name', where name is string literal and number is an integer",
|
||||
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
|
||||
arguments->children = assign_number_child;
|
||||
}
|
||||
|
||||
template <typename DataTypeEnum>
|
||||
static DataTypePtr createExact(const ASTPtr & arguments)
|
||||
{
|
||||
@ -202,6 +229,7 @@ static DataTypePtr createExact(const ASTPtr & arguments)
|
||||
|
||||
using FieldType = typename DataTypeEnum::FieldType;
|
||||
|
||||
autoAssignNumberForEnum(arguments);
|
||||
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
{
|
||||
@ -236,6 +264,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
||||
autoAssignNumberForEnum(arguments);
|
||||
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
{
|
||||
|
@ -50,11 +50,8 @@ public:
|
||||
res0_indexes.resize(cell_count);
|
||||
getRes0Cells(res0_indexes.data());
|
||||
|
||||
auto res = ColumnArray::create(ColumnUInt64::create());
|
||||
|
||||
Array res_indexes;
|
||||
res_indexes.insert(res_indexes.end(), res0_indexes.begin(), res0_indexes.end());
|
||||
res->insert(res_indexes);
|
||||
|
||||
return result_type->createColumnConst(input_rows_count, res_indexes);
|
||||
}
|
||||
|
@ -205,12 +205,8 @@ bool ParallelReadBuffer::nextImpl()
|
||||
void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker)
|
||||
{
|
||||
SCOPE_EXIT({
|
||||
std::lock_guard lock{mutex};
|
||||
--active_working_reader;
|
||||
if (active_working_reader == 0)
|
||||
{
|
||||
readers_done.notify_all();
|
||||
}
|
||||
if (active_working_reader.fetch_sub(1) == 1)
|
||||
active_working_reader.notify_all();
|
||||
});
|
||||
|
||||
try
|
||||
@ -265,8 +261,12 @@ void ParallelReadBuffer::finishAndWait()
|
||||
{
|
||||
emergency_stop = true;
|
||||
|
||||
std::unique_lock lock{mutex};
|
||||
readers_done.wait(lock, [&] { return active_working_reader == 0; });
|
||||
size_t active_readers = active_working_reader.load();
|
||||
while (active_readers != 0)
|
||||
{
|
||||
active_working_reader.wait(active_readers);
|
||||
active_readers = active_working_reader.load();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -135,9 +135,7 @@ private:
|
||||
Segment current_segment;
|
||||
|
||||
size_t max_working_readers;
|
||||
size_t active_working_reader{0};
|
||||
// Triggered when all reader workers are done
|
||||
std::condition_variable readers_done;
|
||||
std::atomic_size_t active_working_reader{0};
|
||||
|
||||
CallbackRunner schedule;
|
||||
|
||||
|
@ -97,6 +97,21 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
type_name_suffix = "SIGNED";
|
||||
else if (ParserKeyword("UNSIGNED").ignore(pos))
|
||||
type_name_suffix = "UNSIGNED";
|
||||
else if (pos->type == TokenType::OpeningRoundBracket)
|
||||
{
|
||||
++pos;
|
||||
if (pos->type != TokenType::Number)
|
||||
return false;
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
if (ParserKeyword("SIGNED").ignore(pos))
|
||||
type_name_suffix = "SIGNED";
|
||||
else if (ParserKeyword("UNSIGNED").ignore(pos))
|
||||
type_name_suffix = "UNSIGNED";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (!type_name_suffix.empty())
|
||||
|
@ -363,9 +363,13 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
|
||||
current_parts.remove(*drop_range_part_name);
|
||||
}
|
||||
|
||||
/// During inserting to queue (insertUnlocked()) we remove part for
|
||||
/// DROP_RANGE only for DROP PART but not for DROP PARTITION.
|
||||
virtual_parts.remove(*drop_range_part_name);
|
||||
|
||||
removeCoveredPartsFromMutations(*drop_range_part_name, /*remove_part = */ true, /*remove_covered_parts = */ false);
|
||||
/// NOTE: we don't need to remove part/covered parts from mutations (removeCoveredPartsFromMutations()) here because:
|
||||
/// - for DROP PART we have this during inserting to queue (see insertUnlocked())
|
||||
/// - for DROP PARTITION we have this in the loop above (when we adding parts to current_parts)
|
||||
}
|
||||
|
||||
if (entry->type == LogEntry::DROP_RANGE)
|
||||
|
@ -5,10 +5,12 @@ import re
|
||||
from typing import Tuple
|
||||
|
||||
from github import Github
|
||||
from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL
|
||||
from pr_info import PRInfo
|
||||
from get_robot_token import get_best_robot_token
|
||||
|
||||
from commit_status_helper import get_commit, post_labels, remove_labels
|
||||
from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL
|
||||
from get_robot_token import get_best_robot_token
|
||||
from pr_info import PRInfo
|
||||
from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS
|
||||
|
||||
NAME = "Run Check (actions)"
|
||||
|
||||
@ -24,70 +26,12 @@ DO_NOT_TEST_LABEL = "do not test"
|
||||
FORCE_TESTS_LABEL = "force tests"
|
||||
SUBMODULE_CHANGED_LABEL = "submodule changed"
|
||||
|
||||
# Individual trusted contirbutors who are not in any trusted organization.
|
||||
# Can be changed in runtime: we will append users that we learned to be in
|
||||
# a trusted org, to save GitHub API calls.
|
||||
TRUSTED_CONTRIBUTORS = {
|
||||
e.lower()
|
||||
for e in [
|
||||
"achimbab",
|
||||
"adevyatova ", # DOCSUP
|
||||
"Algunenano", # Raúl Marín, Tinybird
|
||||
"amosbird",
|
||||
"AnaUvarova", # DOCSUP
|
||||
"anauvarova", # technical writer, Yandex
|
||||
"annvsh", # technical writer, Yandex
|
||||
"atereh", # DOCSUP
|
||||
"azat",
|
||||
"bharatnc", # Newbie, but already with many contributions.
|
||||
"bobrik", # Seasoned contributor, CloudFlare
|
||||
"BohuTANG",
|
||||
"codyrobert", # Flickerbox engineer
|
||||
"cwurm", # Employee
|
||||
"damozhaeva", # DOCSUP
|
||||
"den-crane",
|
||||
"flickerbox-tom", # Flickerbox
|
||||
"gyuton", # technical writer, Yandex
|
||||
"hagen1778", # Roman Khavronenko, seasoned contributor
|
||||
"hczhcz",
|
||||
"hexiaoting", # Seasoned contributor
|
||||
"ildus", # adjust, ex-pgpro
|
||||
"javisantana", # a Spanish ClickHouse enthusiast, ex-Carto
|
||||
"ka1bi4", # DOCSUP
|
||||
"kirillikoff", # DOCSUP
|
||||
"kitaisreal", # Seasoned contributor
|
||||
"kreuzerkrieg",
|
||||
"lehasm", # DOCSUP
|
||||
"michon470", # DOCSUP
|
||||
"MyroTk", # Tester in Altinity
|
||||
"myrrc", # Michael Kot, Altinity
|
||||
"nikvas0",
|
||||
"nvartolomei",
|
||||
"olgarev", # DOCSUP
|
||||
"otrazhenia", # Yandex docs contractor
|
||||
"pdv-ru", # DOCSUP
|
||||
"podshumok", # cmake expert from QRator Labs
|
||||
"s-mx", # Maxim Sabyanin, former employee, present contributor
|
||||
"sevirov", # technical writer, Yandex
|
||||
"spongedu", # Seasoned contributor
|
||||
"taiyang-li",
|
||||
"ucasFL", # Amos Bird's friend
|
||||
"vdimir", # Employee
|
||||
"vzakaznikov",
|
||||
"YiuRULE",
|
||||
"zlobober", # Developer of YT
|
||||
"ilejn", # Arenadata, responsible for Kerberized Kafka
|
||||
"thomoco", # ClickHouse
|
||||
"BoloniniD", # Seasoned contributor, HSE
|
||||
"tonickkozlov", # Cloudflare
|
||||
"tylerhannan", # ClickHouse Employee
|
||||
]
|
||||
}
|
||||
|
||||
MAP_CATEGORY_TO_LABEL = {
|
||||
"New Feature": "pr-feature",
|
||||
"Bug Fix": "pr-bugfix",
|
||||
"Bug Fix (user-visible misbehaviour in official stable or prestable release)": "pr-bugfix",
|
||||
"Bug Fix (user-visible misbehaviour in official "
|
||||
"stable or prestable release)": "pr-bugfix",
|
||||
"Improvement": "pr-improvement",
|
||||
"Performance Improvement": "pr-performance",
|
||||
"Backward Incompatible Change": "pr-backward-incompatible",
|
||||
@ -257,9 +201,7 @@ if __name__ == "__main__":
|
||||
elif SUBMODULE_CHANGED_LABEL in pr_info.labels:
|
||||
pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL)
|
||||
|
||||
print(
|
||||
"change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove)
|
||||
)
|
||||
print(f"change labels: add {pr_labels_to_add}, remove {pr_labels_to_remove}")
|
||||
if pr_labels_to_add:
|
||||
post_labels(gh, pr_info, pr_labels_to_add)
|
||||
|
||||
|
1
tests/ci/workflow_approve_rerun_lambda/__init__.py
Normal file
1
tests/ci/workflow_approve_rerun_lambda/__init__.py
Normal file
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python
|
@ -89,9 +89,11 @@ TRUSTED_CONTRIBUTORS = {
|
||||
"bharatnc", # Newbie, but already with many contributions.
|
||||
"bobrik", # Seasoned contributor, CloudFlare
|
||||
"BohuTANG",
|
||||
"codyrobert", # Flickerbox engineer
|
||||
"cwurm", # Employee
|
||||
"damozhaeva", # DOCSUP
|
||||
"den-crane",
|
||||
"flickerbox-tom", # Flickerbox
|
||||
"gyuton", # DOCSUP
|
||||
"hagen1778", # Roman Khavronenko, seasoned contributor
|
||||
"hczhcz",
|
||||
@ -120,7 +122,10 @@ TRUSTED_CONTRIBUTORS = {
|
||||
"vzakaznikov",
|
||||
"YiuRULE",
|
||||
"zlobober", # Developer of YT
|
||||
"ilejn", # Arenadata, responsible for Kerberized Kafka
|
||||
"thomoco", # ClickHouse
|
||||
"BoloniniD", # Seasoned contributor, HSE
|
||||
"tonickkozlov", # Cloudflare
|
||||
"tylerhannan", # ClickHouse Employee
|
||||
]
|
||||
}
|
||||
|
@ -4,3 +4,5 @@ iphone 1
|
||||
iphone 1
|
||||
|
||||
\N 1
|
||||
a
|
||||
b
|
||||
|
@ -1,2 +1,12 @@
|
||||
select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS;
|
||||
select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS;
|
||||
|
||||
DROP TABLE IF EXISTS auto_assgin_enum;
|
||||
DROP TABLE IF EXISTS auto_assgin_enum1;
|
||||
|
||||
CREATE TABLE auto_assgin_enum (x enum('a', 'b')) ENGINE=MergeTree() order by x;
|
||||
CREATE TABLE auto_assgin_enum1 (x enum('a' = 1, 'b')) ENGINE=MergeTree() order by x; -- { serverError 223 }
|
||||
INSERT INTO auto_assgin_enum VALUES('a'), ('b');
|
||||
select * from auto_assgin_enum;
|
||||
|
||||
DROP TABLE auto_assgin_enum;
|
||||
|
@ -1 +1,5 @@
|
||||
1
|
||||
CREATE TEMPORARY TABLE t3_00841\n(\n `x` UInt32\n)\nENGINE = Memory
|
||||
1
|
||||
CREATE TEMPORARY TABLE t4_00841\n(\n `x` Int32\n)\nENGINE = Memory
|
||||
1
|
||||
|
@ -3,3 +3,13 @@ INSERT INTO t1_00841 VALUES (1);
|
||||
SELECT * FROM t1_00841;
|
||||
|
||||
CREATE TEMPORARY TABLE test.t2_00841 (x UInt8); -- { serverError 442 }
|
||||
|
||||
CREATE TEMPORARY TABLE t3_00841 (x INT(11) UNSIGNED);
|
||||
SHOW CREATE TEMPORARY TABLE t3_00841;
|
||||
INSERT INTO t3_00841 VALUES (1);
|
||||
SELECT * FROM t3_00841;
|
||||
|
||||
CREATE TEMPORARY TABLE t4_00841 (x INT(11) SIGNED);
|
||||
SHOW CREATE TEMPORARY TABLE t4_00841;
|
||||
INSERT INTO t4_00841 VALUES (1);
|
||||
SELECT * FROM t4_00841;
|
||||
|
@ -0,0 +1 @@
|
||||
['x'] 0 ['1','2','3','4','5','6']
|
@ -0,0 +1,22 @@
|
||||
DROP TABLE IF EXISTS t_02267;
|
||||
|
||||
CREATE TABLE t_02267
|
||||
(
|
||||
a Array(String),
|
||||
b UInt32,
|
||||
c Array(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY b
|
||||
SETTINGS index_granularity = 500;
|
||||
|
||||
INSERT INTO t_02267 (b, a, c) SELECT 0, ['x'], ['1','2','3','4','5','6'] FROM numbers(1) ;
|
||||
INSERT INTO t_02267 (b, a, c) SELECT 1, [], ['1','2','3','4','5','6'] FROM numbers(300000);
|
||||
|
||||
OPTIMIZE TABLE t_02267 FINAL;
|
||||
|
||||
SELECT * FROM t_02267 WHERE hasAll(a, ['x'])
|
||||
ORDER BY b DESC
|
||||
SETTINGS max_threads=1, max_block_size=1000;
|
||||
|
||||
DROP TABLE IF EXISTS t_02267;
|
Loading…
Reference in New Issue
Block a user