mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #6278 from yandex/fix_bool_settings_parsing
Fix bool settings parsing
This commit is contained in:
commit
9af4542230
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<merge_tree>
|
||||
<replicated_can_become_leader>0</replicated_can_become_leader>
|
||||
</merge_tree>
|
||||
</yandex>
|
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<merge_tree>
|
||||
<replicated_can_become_leader>FAlse</replicated_can_become_leader>
|
||||
</merge_tree>
|
||||
</yandex>
|
@ -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'
|
@ -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";
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user