Merge pull request #6278 from yandex/fix_bool_settings_parsing

Fix bool settings parsing
This commit is contained in:
alexey-milovidov 2019-08-03 01:55:42 +03:00 committed by GitHub
commit 9af4542230
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 81 additions and 2 deletions

View File

@ -441,6 +441,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_ELF = 464;
extern const int CANNOT_PARSE_DWARF = 465;
extern const int INSECURE_PATH = 466;
extern const int CANNOT_PARSE_BOOL = 467;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -4,6 +4,7 @@
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/FieldVisitors.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteHelpers.h>
@ -26,6 +27,7 @@ namespace ErrorCodes
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_SETTING;
extern const int CANNOT_PARSE_BOOL;
}
@ -63,6 +65,30 @@ void SettingNumber<Type>::set(const String & x)
set(parse<Type>(x));
}
template <>
void SettingNumber<bool>::set(const String & x)
{
if (x.size() == 1)
{
if (x[0] == '0')
set(false);
else if (x[0] == '1')
set(true);
else
throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL);
}
else
{
ReadBufferFromString buf(x);
if (checkStringCaseInsensitive("true", buf))
set(true);
else if (checkStringCaseInsensitive("false", buf))
set(false);
else
throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL);
}
}
template <typename Type>
void SettingNumber<Type>::serialize(WriteBuffer & buf) const
{

View File

@ -0,0 +1,5 @@
<yandex>
<merge_tree>
<replicated_can_become_leader>0</replicated_can_become_leader>
</merge_tree>
</yandex>

View File

@ -0,0 +1,5 @@
<yandex>
<merge_tree>
<replicated_can_become_leader>FAlse</replicated_can_become_leader>
</merge_tree>
</yandex>

View File

@ -0,0 +1,42 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/notleader.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/notleaderignorecase.xml'], with_zookeeper=True)
node3 = cluster.add_instance('node3', with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
for i, node in enumerate((node1, node2)):
node.query(
'''
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table', '{}')
PARTITION BY date ORDER BY id
'''.format(i)
)
with pytest.raises(QueryRuntimeException):
node3.query(
'''
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table', '{}')
PARTITION BY date ORDER BY id SETTINGS replicated_can_become_leader=0sad
'''.format(3)
)
yield cluster
finally:
cluster.shutdown()
def test_can_become_leader(start_cluster):
assert node1.query("select can_become_leader from system.replicas where table = 'test_table'") == '0\n'
assert node2.query("select can_become_leader from system.replicas where table = 'test_table'") == '0\n'

View File

@ -12,7 +12,7 @@ Hello "world", 789 ,2016-01-03
"Hello
world", 100, 2016-01-04,
default,,
default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields --query="INSERT INTO csv FORMAT CSV";
default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";

View File

@ -8,7 +8,7 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=5&send_progress_in_htt
# "grep -v 11" in order to skip extra progress header for 11-th row (for processors pipeline)
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&experimental_use_processors=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | grep -v 11
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1experimental_use_processors=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1&experimental_use_processors=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d
# 'send_progress_in_http_headers' is false by default
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -q 'X-ClickHouse-Progress' && echo 'Fail' || true