Merge pull request #8475 from ClickHouse/aku/logical-error-assert

In debug builds, treat LOGICAL_ERROR as an assertion failure.
This commit is contained in:
alexey-milovidov 2020-01-10 19:58:39 +03:00 committed by GitHub
commit 5e29fe3425
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 51 additions and 45 deletions

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int UNKNOWN_EXCEPTION;
extern const int CANNOT_TRUNCATE_FILE;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
@ -33,6 +34,8 @@ Exception::Exception()
Exception::Exception(const std::string & msg, int code)
: Poco::Exception(msg, code)
{
// In debug builds, treat LOGICAL_ERROR as an assertion failure.
assert(code != ErrorCodes::LOGICAL_ERROR);
}
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)

View File

@ -371,7 +371,7 @@ struct JavaHashUTF16LEImpl
}
if (size % 2 != 0)
throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::LOGICAL_ERROR);
throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::BAD_ARGUMENTS);
UInt32 h = 0;
for (size_t i = 0; i < size; i += 2)

View File

@ -36,6 +36,7 @@ namespace ErrorCodes
{
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
namespace Regexps
@ -205,7 +206,7 @@ namespace MultiRegexps
else
throw Exception(
"Pattern '" + str_patterns[error->expression] + "' failed with error '" + String(error->message),
ErrorCodes::LOGICAL_ERROR);
ErrorCodes::BAD_ARGUMENTS);
}
ProfileEvents::increment(ProfileEvents::RegexpCreated);

View File

@ -18,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
struct FormatImpl
@ -45,11 +46,11 @@ struct FormatImpl
for (UInt64 pos = l; pos < r; pos++)
{
if (!isNumericASCII(description[pos]))
throw Exception("Not a number in curly braces at position " + std::to_string(pos), ErrorCodes::LOGICAL_ERROR);
throw Exception("Not a number in curly braces at position " + std::to_string(pos), ErrorCodes::BAD_ARGUMENTS);
res = res * 10 + description[pos] - '0';
if (res >= argument_threshold)
throw Exception(
"Too big number for arguments, must be at most " + std::to_string(argument_threshold), ErrorCodes::LOGICAL_ERROR);
"Too big number for arguments, must be at most " + std::to_string(argument_threshold), ErrorCodes::BAD_ARGUMENTS);
}
}
@ -114,7 +115,7 @@ struct FormatImpl
}
if (is_open_curly)
throw Exception("Two open curly braces without close one at position " + std::to_string(i), ErrorCodes::LOGICAL_ERROR);
throw Exception("Two open curly braces without close one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
String to_add = String(pattern.data() + start_pos, i - start_pos);
double_brace_removal(to_add);
@ -137,7 +138,7 @@ struct FormatImpl
}
if (!is_open_curly)
throw Exception("Closed curly brace without open one at position " + std::to_string(i), ErrorCodes::LOGICAL_ERROR);
throw Exception("Closed curly brace without open one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
is_open_curly = false;
@ -145,17 +146,17 @@ struct FormatImpl
{
if (is_plain_numbering && !*is_plain_numbering)
throw Exception(
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::LOGICAL_ERROR);
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
is_plain_numbering = true;
if (index_if_plain >= argument_number)
throw Exception("Argument is too big for formatting", ErrorCodes::LOGICAL_ERROR);
throw Exception("Argument is too big for formatting", ErrorCodes::BAD_ARGUMENTS);
*index_positions_ptr = index_if_plain++;
}
else
{
if (is_plain_numbering && *is_plain_numbering)
throw Exception(
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::LOGICAL_ERROR);
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
is_plain_numbering = false;
UInt64 arg;
@ -163,7 +164,7 @@ struct FormatImpl
if (arg >= argument_number)
throw Exception(
"Argument is too big for formatting. Note that indexing starts from zero", ErrorCodes::LOGICAL_ERROR);
"Argument is too big for formatting. Note that indexing starts from zero", ErrorCodes::BAD_ARGUMENTS);
*index_positions_ptr = arg;
}
@ -183,7 +184,7 @@ struct FormatImpl
}
if (is_open_curly)
throw Exception("Last open curly brace is not closed", ErrorCodes::LOGICAL_ERROR);
throw Exception("Last open curly brace is not closed", ErrorCodes::BAD_ARGUMENTS);
String to_add = String(pattern.data() + start_pos, pattern.size() - start_pos);
double_brace_removal(to_add);

View File

@ -96,7 +96,7 @@ DataTypePtr BloomFilter::getPrimitiveType(const DataTypePtr & data_type)
if (!typeid_cast<const DataTypeArray *>(array_type->getNestedType().get()))
return getPrimitiveType(array_type->getNestedType());
else
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
}
if (const auto * nullable_type = typeid_cast<const DataTypeNullable *>(data_type.get()))

View File

@ -23,6 +23,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
}
struct BloomFilterHash
@ -69,7 +70,7 @@ struct BloomFilterHash
unexpected_type = true;
if (unexpected_type)
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
return ColumnConst::create(ColumnUInt64::create(1, hash), 1);
}
@ -82,7 +83,7 @@ struct BloomFilterHash
const auto * array_col = typeid_cast<const ColumnArray *>(column.get());
if (checkAndGetColumn<ColumnNullable>(array_col->getData()))
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
const auto & offsets = array_col->getOffsets();
limit = offsets[pos + limit - 1] - offsets[pos - 1]; /// PaddedPODArray allows access on index -1.
@ -127,7 +128,7 @@ struct BloomFilterHash
else if (which.isFloat64()) getNumberTypeHash<Float64, is_first>(column, vec, pos);
else if (which.isString()) getStringTypeHash<is_first>(column, vec, pos);
else if (which.isFixedString()) getStringTypeHash<is_first>(column, vec, pos);
else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::LOGICAL_ERROR);
else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
}
template <typename Type, bool is_first>

View File

@ -1218,7 +1218,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
}
else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube)
throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::LOGICAL_ERROR);
throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED);
need_second_distinct_pass = query.distinct && pipeline.hasMixedStreams();

View File

@ -316,7 +316,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
if (insert_it == metadata.indices.indices.end())
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
ErrorCodes::LOGICAL_ERROR);
ErrorCodes::BAD_ARGUMENTS);
++insert_it;
}
@ -338,7 +338,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
if (if_exists)
return;
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.",
ErrorCodes::LOGICAL_ERROR);
ErrorCodes::BAD_ARGUMENTS);
}
metadata.indices.indices.erase(erase_it);
@ -378,7 +378,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
if (if_exists)
return;
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
ErrorCodes::LOGICAL_ERROR);
ErrorCodes::BAD_ARGUMENTS);
}
metadata.constraints.constraints.erase(erase_it);
}

View File

@ -1152,7 +1152,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
if (!canReplacePartition(src_part))
throw Exception(
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
ErrorCodes::LOGICAL_ERROR);
ErrorCodes::BAD_ARGUMENTS);
/// This will generate unique name in scope of current server process.
Int64 temp_index = insert_increment.get();

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'none';
select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 2, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 3, ['a1c']) from system.numbers limit 5; -- { serverError 49 }
select 1 = multiFuzzyMatchAny('abc', 4, ['a1c']) from system.numbers limit 5; -- { serverError 49 }
select 1 = multiFuzzyMatchAny('abc', 3, ['a1c']) from system.numbers limit 5; -- { serverError 36 }
select 1 = multiFuzzyMatchAny('abc', 4, ['a1c']) from system.numbers limit 5; -- { serverError 36 }
select 1 = multiFuzzyMatchAny('leftabcright', 1, ['a1c']) from system.numbers limit 5;
@ -14,7 +14,7 @@ select 0 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 3, ['^hello.*world$']);
select 1 = multiFuzzyMatchAny('hello some world', 10, ['^hello.*world$']); -- { serverError 49 }
select 1 = multiFuzzyMatchAny('hello some world', 10, ['^hello.*world$']); -- { serverError 36 }
select 1 = multiFuzzyMatchAny('hello some world', -1, ['^hello.*world$']); -- { serverError 43 }
select 1 = multiFuzzyMatchAny('hello some world', 10000000000, ['^hello.*world$']); -- { serverError 44 }
select 1 = multiFuzzyMatchAny('http://hyperscan_is_nice.ru/st', 2, ['http://hyperscan_is_nice.ru/(st\\d\\d$|st\\d\\d\\.|st1[0-4]\\d|st150|st\\d$|gl|rz|ch)']);

View File

@ -16,25 +16,25 @@ select 100 = length(format(concat((select arrayStringConcat(arrayMap(x ->'}', ra
select format('', 'first');
select concat('third', 'first', 'second')=format('{2}{0}{1}', 'first', 'second', 'third');
select format('{', ''); -- { serverError 49 }
select format('{{}', ''); -- { serverError 49 }
select format('{ {}', ''); -- { serverError 49 }
select format('}', ''); -- { serverError 49 }
select format('{', ''); -- { serverError 36 }
select format('{{}', ''); -- { serverError 36 }
select format('{ {}', ''); -- { serverError 36 }
select format('}', ''); -- { serverError 36 }
select format('{{', '');
select format('{}}', ''); -- { serverError 49 }
select format('{}}', ''); -- { serverError 36 }
select format('}}', '');
select format('{2 }', ''); -- { serverError 49 }
select format('{}{}{}{}{}{} }{}', '', '', '', '', '', '', ''); -- { serverError 49 }
select format('{sometext}', ''); -- { serverError 49 }
select format('{\0sometext}', ''); -- { serverError 49 }
select format('{1023}', ''); -- { serverError 49 }
select format('{10000000000000000000000000000000000000000000000000}', ''); -- { serverError 49 }
select format('{} {0}', '', ''); -- { serverError 49 }
select format('{0} {}', '', ''); -- { serverError 49 }
select format('Hello {} World {} {}{}', 'first', 'second', 'third') from system.numbers limit 2; -- { serverError 49 }
select format('Hello {0} World {1} {2}{3}', 'first', 'second', 'third') from system.numbers limit 2; -- { serverError 49 }
select format('{2 }', ''); -- { serverError 36 }
select format('{}{}{}{}{}{} }{}', '', '', '', '', '', '', ''); -- { serverError 36 }
select format('{sometext}', ''); -- { serverError 36 }
select format('{\0sometext}', ''); -- { serverError 36 }
select format('{1023}', ''); -- { serverError 36 }
select format('{10000000000000000000000000000000000000000000000000}', ''); -- { serverError 36 }
select format('{} {0}', '', ''); -- { serverError 36 }
select format('{0} {}', '', ''); -- { serverError 36 }
select format('Hello {} World {} {}{}', 'first', 'second', 'third') from system.numbers limit 2; -- { serverError 36 }
select format('Hello {0} World {1} {2}{3}', 'first', 'second', 'third') from system.numbers limit 2; -- { serverError 36 }
select 50 = length(format((select arrayStringConcat(arrayMap(x ->'{', range(101)))), '')); -- { serverError 49 }
select 50 = length(format((select arrayStringConcat(arrayMap(x ->'{', range(101)))), '')); -- { serverError 36 }
select format('{}{}{}', materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc';
select format('{}{}{}', materialize(toFixedString('a', 1)), materialize('b'), materialize(toFixedString('c', 1))) == 'abc';

View File

@ -1,6 +1,6 @@
SELECT 11 AS n GROUP BY n WITH TOTALS;
SELECT 12 AS n GROUP BY n WITH ROLLUP;
SELECT 13 AS n GROUP BY n WITH CUBE;
SELECT 1 AS n WITH TOTALS; -- { serverError 49 }
SELECT 1 AS n WITH ROLLUP; -- { serverError 49 }
SELECT 1 AS n WITH CUBE; -- { serverError 49 }
SELECT 1 AS n WITH TOTALS; -- { serverError 48 }
SELECT 1 AS n WITH ROLLUP; -- { serverError 48 }
SELECT 1 AS n WITH CUBE; -- { serverError 48 }

View File

@ -24,7 +24,7 @@ CREATE TABLE non_mixed_granularity_adaptive_table AS test.hits;
INSERT INTO non_mixed_granularity_adaptive_table SELECT * FROM test.hits LIMIT 10;
ALTER TABLE non_mixed_granularity_adaptive_table REPLACE PARTITION 201403 FROM test.hits; -- { serverError 49 }
ALTER TABLE non_mixed_granularity_adaptive_table REPLACE PARTITION 201403 FROM test.hits; -- { serverError 36 }
DROP TABLE IF EXISTS non_mixed_granularity_adaptive_table;
@ -35,7 +35,7 @@ CREATE TABLE non_mixed_granularity_non_adaptive_table (`WatchID` UInt64, `JavaEn
INSERT INTO non_mixed_granularity_non_adaptive_table SELECT * FROM test.hits LIMIT 10;
-- after optimize mixed_granularity_table will have .mrk2 parts
ALTER TABLE non_mixed_granularity_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 49 }
ALTER TABLE non_mixed_granularity_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 36 }
DROP TABLE IF EXISTS non_mixed_granularity_non_adaptive_table;
@ -46,7 +46,7 @@ CREATE TABLE mixed_granularity_strictly_non_adaptive_table (`WatchID` UInt64, `J
INSERT INTO mixed_granularity_strictly_non_adaptive_table SELECT * FROM test.hits LIMIT 10;
ALTER TABLE mixed_granularity_strictly_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 49 }
ALTER TABLE mixed_granularity_strictly_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 36 }
DROP TABLE IF EXISTS mixed_granularity_table;