Merge branch 'master' into clang-tidy-8

This commit is contained in:
Alexey Milovidov 2020-04-22 07:54:27 +03:00
commit b9b5036749
26 changed files with 218 additions and 22 deletions

View File

@ -100,7 +100,7 @@ function run_tests
# changes.
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
rm -v "$x" ||:
touch "$x"

View File

@ -562,7 +562,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
format_schema_path.createDirectories();
/// 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);
size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio;

View File

@ -90,6 +90,23 @@
<!-- Maximum number of 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
correct maximum value. -->
<!-- <max_open_files>262144</max_open_files> -->

View File

@ -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_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, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \

View File

@ -84,7 +84,7 @@ namespace
try
{
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();
database.attachDictionary(create_query.table, DictionaryAttachInfo{query, config, modification_time});
}

View File

@ -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);
@ -438,7 +438,7 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
AutoPtr<Poco::XML::Element> database_element(xml_document->createElement("database"));
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);
AutoPtr<Element> structure_element(xml_document->createElement("structure"));

View File

@ -10,5 +10,5 @@ using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfigurati
/// Convert dictionary AST to Poco::AbstractConfiguration
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
/// Can throw exception if query is ill-formed
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const std::string & database_ = "");
}

View File

@ -1000,6 +1000,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
Collection arr;
ParserLiteral literal_p;
ParserCollectionOfLiterals<Collection> collection_p(opening_bracket, closing_bracket);
++pos;
while (pos.isValid())
@ -1032,7 +1033,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & 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;
arr.push_back(literal_node->as<ASTLiteral &>().value);

View File

@ -105,6 +105,14 @@ static void fillLiteralInfo(DataTypes & nested_types, LiteralInfo & info)
{
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
throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug",
ErrorCodes::LOGICAL_ERROR);

View File

@ -126,7 +126,7 @@ void TabSeparatedRowInputFormat::fillUnreadColumnsWithDefaults(MutableColumns &
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,
/// so, if format has header,

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

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

View File

@ -9,7 +9,8 @@
<substitution>
<name>aggregationscale_all</name>
<values>
<value>1111111</value>
<value>5555555</value>
<value>555555</value>
<value>111111</value>
<value>11111</value>
</values>
@ -24,8 +25,8 @@
<substitution>
<name>aggregationscale_big</name>
<values>
<value>1111111</value>
<value>111111</value>
<value>5555555</value>
<value>555555</value>
</values>
</substitution>
</substitutions>

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,4 @@
0 NOT_LOADED
0 LOADED
10
1 LOADED

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

View File

@ -0,0 +1,4 @@
0 NOT_LOADED
0 LOADED
10
1 LOADED

View File

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

View File

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

View File

@ -0,0 +1,2 @@
1 2 3
4 5 6

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

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