mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #41441 from ClickHouse/pod-array-overflow
Fix overflow in `PODArray`
This commit is contained in:
commit
57a1f47972
@ -115,7 +115,13 @@ protected:
|
||||
}
|
||||
|
||||
/// Minimum amount of memory to allocate for num_elements, including padding.
|
||||
static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; } /// NOLINT
|
||||
static size_t minimum_memory_for_elements(size_t num_elements)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_add_overflow(byte_size(num_elements), pad_left + pad_right, &amount))
|
||||
throw Exception("Amount of memory requested to allocate is more than allowed", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
return amount;
|
||||
}
|
||||
|
||||
void alloc_for_num_elements(size_t num_elements) /// NOLINT
|
||||
{
|
||||
|
@ -20,8 +20,13 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||
}
|
||||
|
||||
static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
|
||||
static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40;
|
||||
|
||||
|
||||
void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
const Array & a = field.get<const Array &>();
|
||||
@ -125,7 +130,12 @@ namespace
|
||||
{
|
||||
ColumnArray::Offset current_size = 0;
|
||||
readIntBinary(current_size, istr);
|
||||
current_offset += current_size;
|
||||
|
||||
if (unlikely(current_size > MAX_ARRAY_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size);
|
||||
if (unlikely(__builtin_add_overflow(current_offset, current_size, ¤t_offset)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Deserialization of array offsets will lead to overflow");
|
||||
|
||||
offset_values[i] = current_offset;
|
||||
++i;
|
||||
}
|
||||
@ -348,6 +358,9 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams(
|
||||
throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
||||
size_t nested_limit = last_offset - nested_column->size();
|
||||
|
||||
if (unlikely(nested_limit > MAX_ARRAYS_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array sizes are too large: {}", nested_limit);
|
||||
|
||||
/// Adjust value size hint. Divide it to the average array size.
|
||||
settings.avg_value_size_hint = nested_limit ? settings.avg_value_size_hint / nested_limit * offset_values.size() : 0;
|
||||
|
||||
|
@ -24,6 +24,8 @@ namespace ErrorCodes
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30;
|
||||
|
||||
void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
const String & s = field.get<const String &>();
|
||||
@ -85,8 +87,17 @@ void SerializationFixedString::deserializeBinaryBulk(IColumn & column, ReadBuffe
|
||||
ColumnFixedString::Chars & data = typeid_cast<ColumnFixedString &>(column).getChars();
|
||||
|
||||
size_t initial_size = data.size();
|
||||
size_t max_bytes = limit * n;
|
||||
data.resize(initial_size + max_bytes);
|
||||
size_t max_bytes;
|
||||
size_t new_data_size;
|
||||
|
||||
if (unlikely(__builtin_mul_overflow(limit, n, &max_bytes)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Deserializing FixedString will lead to overflow");
|
||||
if (unlikely(max_bytes > MAX_STRINGS_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large sizes of FixedString to deserialize: {}", max_bytes);
|
||||
if (unlikely(__builtin_add_overflow(initial_size, max_bytes, &new_data_size)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Deserializing FixedString will lead to overflow");
|
||||
|
||||
data.resize(new_data_size);
|
||||
size_t read_bytes = istr.readBig(reinterpret_cast<char *>(&data[initial_size]), max_bytes);
|
||||
|
||||
if (read_bytes % n != 0)
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <emmintrin.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -63,9 +63,10 @@ ColumnsDescription readSchemaFromFormat(
|
||||
{
|
||||
names_and_types = external_schema_reader->readSchema();
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
catch (Exception & e)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, e.message());
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name))
|
||||
@ -85,6 +86,12 @@ ColumnsDescription readSchemaFromFormat(
|
||||
break;
|
||||
is_eof = buf->eof();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format(
|
||||
"Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
auto exception_message = getCurrentExceptionMessage(false);
|
||||
@ -136,7 +143,21 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
|
||||
if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode()))
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, exception_message);
|
||||
{
|
||||
try
|
||||
{
|
||||
throw;
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, exception_message);
|
||||
}
|
||||
}
|
||||
|
||||
exception_messages += "\n" + exception_message;
|
||||
}
|
||||
|
@ -8,5 +8,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
cp $CUR_DIR/data_orc/corrupted.orc $USER_FILES_PATH/
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="select * from file('corrupted.orc')" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="select * from file('corrupted.orc')" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-parallel
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -12,7 +12,6 @@ DATA_FILE=$USER_FILES_PATH/$FILE_NAME
|
||||
cp $CUR_DIR/data_parquet_bad_column/metadata_0.parquet $DATA_FILE
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet)" 2>&1 | grep -qF "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet)" 2>&1 | grep -qF "Cannot extract table structure" && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1"
|
||||
$CLICKHOUSE_CLIENT -q "select count(*) from file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1"
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: no-fasttest, no-parallel
|
||||
|
||||
insert into function file('02267_data2.jsonl') select NULL as x;
|
||||
insert into function file('02267_data3.jsonl') select * from numbers(0);
|
||||
insert into function file('02267_data4.jsonl') select 1 as x;
|
||||
@ -7,4 +8,4 @@ select * from file('02267_data*.jsonl') order by x;
|
||||
insert into function file('02267_data1.jsonl', 'TSV') select 1 as x;
|
||||
insert into function file('02267_data1.jsonl', 'TSV') select [1,2,3] as x;
|
||||
|
||||
select * from file('02267_data*.jsonl') settings schema_inference_use_cache_for_file=0; --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
select * from file('02267_data*.jsonl') settings schema_inference_use_cache_for_file=0; --{serverError INCORRECT_DATA}
|
||||
|
@ -1,6 +1,7 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
insert into function file('02268_data.jsonl', 'TSV') select 1;
|
||||
select * from file('02268_data.jsonl'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
select * from file('02268_data.jsonl'); --{serverError 117}
|
||||
|
||||
insert into function file('02268_data.jsonCompactEachRow', 'TSV') select 1;
|
||||
select * from file('02268_data.jsonCompactEachRow'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
|
||||
select * from file('02268_data.jsonCompactEachRow'); --{serverError 117}
|
||||
|
@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mys
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32)') settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'EMPTY_DATA_PASSED' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
echo "dump2"
|
||||
@ -146,4 +146,3 @@ $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_my
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'"
|
||||
|
||||
rm $USER_FILES_PATH/dump*.sql
|
||||
|
||||
|
@ -88,6 +88,4 @@ echo '
|
||||
}
|
||||
' > $DATA_FILE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns) settings input_format_max_rows_to_read_for_schema_inference=3" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns) settings input_format_max_rows_to_read_for_schema_inference=3" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
|
@ -15,11 +15,11 @@ mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR
|
||||
cp -r $CLIENT_SCHEMADIR/02327_* $SCHEMADIR/$SERVER_SCHEMADIR/
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1";
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1";
|
||||
|
1
tests/queries/0_stateless/02424_pod_array_overflow.sql
Normal file
1
tests/queries/0_stateless/02424_pod_array_overflow.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, '\x02\x02\x02\x6b\x30\x1a\x4d\x61\x70\x28\x46\x69\x78\x65\x64\x53\x74\x72\x69\x6e\x67\x28\x31\x29\x2c\x20\x49\x6e\x74\x36\x34\x29\x01\x00\x00\x00\x00\x00\x00\x00\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x7f\x00\x7f\xff\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x64\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xcf\x31\x3f\x56\x69\x11\x89\x25'); -- { serverError 128 }
|
1
tests/queries/0_stateless/02426_pod_array_overflow_2.sql
Normal file
1
tests/queries/0_stateless/02426_pod_array_overflow_2.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, 'k0\x23Array(Tuple(FixedString(1), Int64))\0\0\0\0\0\0\0<><30><EFBFBD><EFBFBD><EFBFBD>\0<><30><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>\0<30>\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0d\0\0\0\0\0\0\0\0\0\0\0\0\0<>1?Vi<11>%'); -- { serverError 128 }
|
1
tests/queries/0_stateless/02426_pod_array_overflow_3.sql
Normal file
1
tests/queries/0_stateless/02426_pod_array_overflow_3.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, '\x01\x01\x01x\x0CArray(UInt8)\x01\x00\xBD\xEF\xBF\xBD\xEF\xBF\xBD\xEF'); -- { serverError 128 }
|
Loading…
Reference in New Issue
Block a user