mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 02:52:13 +00:00
Merge branch 'master' into clang-tidy-8
This commit is contained in:
commit
b9b5036749
@ -100,7 +100,7 @@ function run_tests
|
|||||||
# changes.
|
# changes.
|
||||||
test_prefix=$([ "$PR_TO_TEST" == "0" ] && echo left || echo right)/performance
|
test_prefix=$([ "$PR_TO_TEST" == "0" ] && echo left || echo right)/performance
|
||||||
|
|
||||||
for x in {test-times,skipped-tests}.tsv
|
for x in {test-times,skipped-tests,wall-clock-times}.tsv
|
||||||
do
|
do
|
||||||
rm -v "$x" ||:
|
rm -v "$x" ||:
|
||||||
touch "$x"
|
touch "$x"
|
||||||
|
@ -562,7 +562,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
format_schema_path.createDirectories();
|
format_schema_path.createDirectories();
|
||||||
|
|
||||||
/// Limit on total memory usage
|
/// Limit on total memory usage
|
||||||
size_t max_server_memory_usage = settings.max_server_memory_usage;
|
size_t max_server_memory_usage = config().getUInt64("max_server_memory_usage", 0);
|
||||||
|
|
||||||
double max_server_memory_usage_to_ram_ratio = config().getDouble("max_server_memory_usage_to_ram_ratio", 0.9);
|
double max_server_memory_usage_to_ram_ratio = config().getDouble("max_server_memory_usage_to_ram_ratio", 0.9);
|
||||||
size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio;
|
size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio;
|
||||||
|
@ -90,6 +90,23 @@
|
|||||||
<!-- Maximum number of concurrent queries. -->
|
<!-- Maximum number of concurrent queries. -->
|
||||||
<max_concurrent_queries>100</max_concurrent_queries>
|
<max_concurrent_queries>100</max_concurrent_queries>
|
||||||
|
|
||||||
|
<!-- Maximum memory usage (resident set size) for server process.
|
||||||
|
Zero value or unset means default. Default is "max_server_memory_usage_to_ram_ratio" of available physical RAM.
|
||||||
|
If the value is larger than "max_server_memory_usage_to_ram_ratio" of available physical RAM, it will be cut down.
|
||||||
|
|
||||||
|
The constraint is checked on query execution time.
|
||||||
|
If a query tries to allocate memory and the current memory usage plus allocation is greater
|
||||||
|
than specified threshold, exception will be thrown.
|
||||||
|
|
||||||
|
It is not practical to set this constraint to small values like just a few gigabytes,
|
||||||
|
because memory allocator will keep this amount of memory in caches and the server will deny service of queries.
|
||||||
|
-->
|
||||||
|
<max_server_memory_usage>0</max_server_memory_usage>
|
||||||
|
|
||||||
|
<!-- On memory constrained environments you may have to set this to value larger than 1.
|
||||||
|
-->
|
||||||
|
<max_server_memory_usage_to_ram_ratio>0.9</max_server_memory_usage_to_ram_ratio>
|
||||||
|
|
||||||
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
|
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
|
||||||
correct maximum value. -->
|
correct maximum value. -->
|
||||||
<!-- <max_open_files>262144</max_open_files> -->
|
<!-- <max_open_files>262144</max_open_files> -->
|
||||||
|
@ -339,7 +339,6 @@ struct Settings : public SettingsCollection<Settings>
|
|||||||
\
|
\
|
||||||
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
|
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
|
||||||
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
|
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
|
||||||
M(SettingUInt64, max_server_memory_usage, 0, "Maximum memory usage for server. Only has meaning at server startup. It can be specified only for default profile.", 0) \
|
|
||||||
M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will collect the allocating stack trace. The minimal effective step is 4 MiB (less values will work as clamped to 4 MiB). Zero means disabled memory profiler.", 0) \
|
M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will collect the allocating stack trace. The minimal effective step is 4 MiB (less values will work as clamped to 4 MiB). Zero means disabled memory profiler.", 0) \
|
||||||
\
|
\
|
||||||
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
|
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
|
||||||
|
@ -84,7 +84,7 @@ namespace
|
|||||||
try
|
try
|
||||||
{
|
{
|
||||||
Poco::File meta_file(metadata_path);
|
Poco::File meta_file(metadata_path);
|
||||||
auto config = getDictionaryConfigurationFromAST(create_query);
|
auto config = getDictionaryConfigurationFromAST(create_query, database.getDatabaseName());
|
||||||
time_t modification_time = meta_file.getLastModified().epochTime();
|
time_t modification_time = meta_file.getLastModified().epochTime();
|
||||||
database.attachDictionary(create_query.table, DictionaryAttachInfo{query, config, modification_time});
|
database.attachDictionary(create_query.table, DictionaryAttachInfo{query, config, modification_time});
|
||||||
}
|
}
|
||||||
|
@ -420,7 +420,7 @@ void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query)
|
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const std::string & database_)
|
||||||
{
|
{
|
||||||
checkAST(query);
|
checkAST(query);
|
||||||
|
|
||||||
@ -438,7 +438,7 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
|
|||||||
|
|
||||||
AutoPtr<Poco::XML::Element> database_element(xml_document->createElement("database"));
|
AutoPtr<Poco::XML::Element> database_element(xml_document->createElement("database"));
|
||||||
current_dictionary->appendChild(database_element);
|
current_dictionary->appendChild(database_element);
|
||||||
AutoPtr<Text> database(xml_document->createTextNode(query.database));
|
AutoPtr<Text> database(xml_document->createTextNode(!database_.empty() ? database_ : query.database));
|
||||||
database_element->appendChild(database);
|
database_element->appendChild(database);
|
||||||
|
|
||||||
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
|
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
|
||||||
|
@ -10,5 +10,5 @@ using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfigurati
|
|||||||
/// Convert dictionary AST to Poco::AbstractConfiguration
|
/// Convert dictionary AST to Poco::AbstractConfiguration
|
||||||
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
|
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
|
||||||
/// Can throw exception if query is ill-formed
|
/// Can throw exception if query is ill-formed
|
||||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
|
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const std::string & database_ = "");
|
||||||
}
|
}
|
||||||
|
@ -1000,6 +1000,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
|
|||||||
|
|
||||||
Collection arr;
|
Collection arr;
|
||||||
ParserLiteral literal_p;
|
ParserLiteral literal_p;
|
||||||
|
ParserCollectionOfLiterals<Collection> collection_p(opening_bracket, closing_bracket);
|
||||||
|
|
||||||
++pos;
|
++pos;
|
||||||
while (pos.isValid())
|
while (pos.isValid())
|
||||||
@ -1032,7 +1033,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
|
|||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr literal_node;
|
ASTPtr literal_node;
|
||||||
if (!literal_p.parse(pos, literal_node, expected))
|
if (!literal_p.parse(pos, literal_node, expected) && !collection_p.parse(pos, literal_node, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
arr.push_back(literal_node->as<ASTLiteral &>().value);
|
arr.push_back(literal_node->as<ASTLiteral &>().value);
|
||||||
|
@ -105,6 +105,14 @@ static void fillLiteralInfo(DataTypes & nested_types, LiteralInfo & info)
|
|||||||
{
|
{
|
||||||
field_type = Field::Types::String;
|
field_type = Field::Types::String;
|
||||||
}
|
}
|
||||||
|
else if (type_info.isArray())
|
||||||
|
{
|
||||||
|
field_type = Field::Types::Array;
|
||||||
|
}
|
||||||
|
else if (type_info.isTuple())
|
||||||
|
{
|
||||||
|
field_type = Field::Types::Tuple;
|
||||||
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug",
|
throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug",
|
||||||
ErrorCodes::LOGICAL_ERROR);
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
@ -126,7 +126,7 @@ void TabSeparatedRowInputFormat::fillUnreadColumnsWithDefaults(MutableColumns &
|
|||||||
|
|
||||||
void TabSeparatedRowInputFormat::readPrefix()
|
void TabSeparatedRowInputFormat::readPrefix()
|
||||||
{
|
{
|
||||||
if (with_names || with_types)
|
if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8())
|
||||||
{
|
{
|
||||||
/// In this format, we assume that column name or type cannot contain BOM,
|
/// In this format, we assume that column name or type cannot contain BOM,
|
||||||
/// so, if format has header,
|
/// so, if format has header,
|
||||||
|
0
tests/integration/test_ddl_alter_query/__init__.py
Normal file
0
tests/integration/test_ddl_alter_query/__init__.py
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
<yandex>
|
||||||
|
<remote_servers>
|
||||||
|
<test_cluster>
|
||||||
|
<shard>
|
||||||
|
<internal_replication>true</internal_replication>
|
||||||
|
<replica>
|
||||||
|
<host>node1</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
<replica>
|
||||||
|
<host>node2</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
</shard>
|
||||||
|
<shard>
|
||||||
|
<internal_replication>true</internal_replication>
|
||||||
|
<replica>
|
||||||
|
<host>node3</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
<replica>
|
||||||
|
<host>node4</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
</shard>
|
||||||
|
</test_cluster>
|
||||||
|
</remote_servers>
|
||||||
|
</yandex>
|
47
tests/integration/test_ddl_alter_query/test.py
Normal file
47
tests/integration/test_ddl_alter_query/test.py
Normal file
@ -0,0 +1,47 @@
|
|||||||
|
import pytest
|
||||||
|
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
|
||||||
|
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||||
|
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||||
|
node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||||
|
node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
|
||||||
|
for i, node in enumerate([node1, node2]):
|
||||||
|
node.query("CREATE DATABASE testdb")
|
||||||
|
node.query('''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table1', '{}') ORDER BY id;'''.format(i))
|
||||||
|
for i, node in enumerate([node3, node4]):
|
||||||
|
node.query("CREATE DATABASE testdb")
|
||||||
|
node.query('''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table2', '{}') ORDER BY id;'''.format(i))
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def test_alter(started_cluster):
|
||||||
|
node1.query("INSERT INTO testdb.test_table SELECT number, toString(number) FROM numbers(100)")
|
||||||
|
node3.query("INSERT INTO testdb.test_table SELECT number, toString(number) FROM numbers(100)")
|
||||||
|
node2.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
node4.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
|
||||||
|
node1.query("ALTER TABLE testdb.test_table ON CLUSTER test_cluster ADD COLUMN somecolumn UInt8 AFTER val", settings={"replication_alter_partitions_sync": "2"})
|
||||||
|
|
||||||
|
node1.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
node2.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
node3.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
node4.query("SYSTEM SYNC REPLICA testdb.test_table")
|
||||||
|
|
||||||
|
assert node1.query("SELECT somecolumn FROM testdb.test_table LIMIT 1") == "0\n"
|
||||||
|
assert node2.query("SELECT somecolumn FROM testdb.test_table LIMIT 1") == "0\n"
|
||||||
|
assert node3.query("SELECT somecolumn FROM testdb.test_table LIMIT 1") == "0\n"
|
||||||
|
assert node4.query("SELECT somecolumn FROM testdb.test_table LIMIT 1") == "0\n"
|
@ -9,7 +9,8 @@
|
|||||||
<substitution>
|
<substitution>
|
||||||
<name>aggregationscale_all</name>
|
<name>aggregationscale_all</name>
|
||||||
<values>
|
<values>
|
||||||
<value>1111111</value>
|
<value>5555555</value>
|
||||||
|
<value>555555</value>
|
||||||
<value>111111</value>
|
<value>111111</value>
|
||||||
<value>11111</value>
|
<value>11111</value>
|
||||||
</values>
|
</values>
|
||||||
@ -24,8 +25,8 @@
|
|||||||
<substitution>
|
<substitution>
|
||||||
<name>aggregationscale_big</name>
|
<name>aggregationscale_big</name>
|
||||||
<values>
|
<values>
|
||||||
<value>1111111</value>
|
<value>5555555</value>
|
||||||
<value>111111</value>
|
<value>555555</value>
|
||||||
</values>
|
</values>
|
||||||
</substitution>
|
</substitution>
|
||||||
</substitutions>
|
</substitutions>
|
||||||
|
File diff suppressed because one or more lines are too long
@ -0,0 +1,4 @@
|
|||||||
|
0 NOT_LOADED
|
||||||
|
0 LOADED
|
||||||
|
10
|
||||||
|
1 LOADED
|
23
tests/queries/0_stateless/01254_dict_create_without_db.sql
Normal file
23
tests/queries/0_stateless/01254_dict_create_without_db.sql
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
DROP DATABASE IF EXISTS dict_db_01254;
|
||||||
|
CREATE DATABASE dict_db_01254;
|
||||||
|
USE dict_db_01254;
|
||||||
|
|
||||||
|
CREATE TABLE dict_data (key UInt64, val UInt64) Engine=Memory();
|
||||||
|
CREATE DICTIONARY dict
|
||||||
|
(
|
||||||
|
key UInt64 DEFAULT 0,
|
||||||
|
val UInt64 DEFAULT 10
|
||||||
|
)
|
||||||
|
PRIMARY KEY key
|
||||||
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01254'))
|
||||||
|
LIFETIME(MIN 0 MAX 0)
|
||||||
|
LAYOUT(FLAT());
|
||||||
|
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
SYSTEM RELOAD DICTIONARY dict_db_01254.dict;
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
SELECT dictGetUInt64('dict_db_01254.dict', 'val', toUInt64(0));
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
|
||||||
|
USE system;
|
||||||
|
DROP DATABASE dict_db_01254;
|
@ -0,0 +1,4 @@
|
|||||||
|
0 NOT_LOADED
|
||||||
|
0 LOADED
|
||||||
|
10
|
||||||
|
1 LOADED
|
@ -0,0 +1,24 @@
|
|||||||
|
DROP DATABASE IF EXISTS dict_db_01254;
|
||||||
|
CREATE DATABASE dict_db_01254;
|
||||||
|
|
||||||
|
CREATE TABLE dict_db_01254.dict_data (key UInt64, val UInt64) Engine=Memory();
|
||||||
|
CREATE DICTIONARY dict_db_01254.dict
|
||||||
|
(
|
||||||
|
key UInt64 DEFAULT 0,
|
||||||
|
val UInt64 DEFAULT 10
|
||||||
|
)
|
||||||
|
PRIMARY KEY key
|
||||||
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01254'))
|
||||||
|
LIFETIME(MIN 0 MAX 0)
|
||||||
|
LAYOUT(FLAT());
|
||||||
|
|
||||||
|
DETACH DATABASE dict_db_01254;
|
||||||
|
ATTACH DATABASE dict_db_01254;
|
||||||
|
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
SYSTEM RELOAD DICTIONARY dict_db_01254.dict;
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
SELECT dictGetUInt64('dict_db_01254.dict', 'val', toUInt64(0));
|
||||||
|
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
|
||||||
|
|
||||||
|
DROP DATABASE dict_db_01254;
|
@ -0,0 +1,5 @@
|
|||||||
|
SELECT * FROM generateRandom('i8', 1, 10, 10); -- { serverError 62 }
|
||||||
|
SELECT * FROM generateRandom; -- { serverError 60 }
|
||||||
|
SELECT * FROM generateRandom(); -- { serverError 42 }
|
||||||
|
SELECT * FROM generateRandom('i8 UInt8', 1, 10, 10, 10, 10); -- { serverError 42 }
|
||||||
|
SELECT * FROM generateRandom('', 1, 10, 10); -- { serverError 62 }
|
File diff suppressed because one or more lines are too long
2
tests/queries/0_stateless/01258_bom_tsv.reference
Normal file
2
tests/queries/0_stateless/01258_bom_tsv.reference
Normal file
@ -0,0 +1,2 @@
|
|||||||
|
1 2 3
|
||||||
|
4 5 6
|
14
tests/queries/0_stateless/01258_bom_tsv.sh
Executable file
14
tests/queries/0_stateless/01258_bom_tsv.sh
Executable file
@ -0,0 +1,14 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
# BOM can be parsed if TSV format has first column that cannot contain arbitrary binary data (such as integer)
|
||||||
|
# In contrast, BOM cannot be parsed if the first column in String as it can contain arbitrary binary data.
|
||||||
|
|
||||||
|
echo 'DROP TABLE IF EXISTS bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
|
||||||
|
echo 'CREATE TABLE bom (a UInt8, b UInt8, c UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
|
||||||
|
echo -ne '1\t2\t3\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+bom+FORMAT+TSV" --data-binary @-
|
||||||
|
echo -ne '\xEF\xBB\xBF4\t5\t6\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+bom+FORMAT+TSV" --data-binary @-
|
||||||
|
echo 'SELECT * FROM bom ORDER BY a' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
|
||||||
|
echo 'DROP TABLE bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
|
1
tests/queries/0_stateless/01258_wrong_cast_filimonov.sql
Normal file
1
tests/queries/0_stateless/01258_wrong_cast_filimonov.sql
Normal file
@ -0,0 +1 @@
|
|||||||
|
create table x( id UInt64, t AggregateFunction(argMax, Enum8('<Empty>' = -1, 'Male' = 1, 'Female' = 2), UInt64) DEFAULT arrayReduce('argMaxState', ['cast(-1, \'Enum8(\'<Empty>\' = -1, \'Male\' = 1, \'Female\' = 2)'], [toUInt64(0)]) ) Engine=MergeTree ORDER BY id; -- { serverError 70 }
|
Loading…
Reference in New Issue
Block a user