mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
commit
5e29fe3425
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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()))
|
||||
|
@ -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>
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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)']);
|
||||
|
@ -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';
|
||||
|
@ -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 }
|
||||
|
@ -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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user