Merge pull request #67324 from ClickHouse/fix-json-recursion

Fix stack overflow in too deep JSON
This commit is contained in:
Alexey Milovidov 2024-08-04 13:21:44 +00:00 committed by GitHub
commit 06f83022da
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 50 additions and 17 deletions

View File

@ -1127,6 +1127,7 @@ class IColumn;
M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \
M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \
M(Bool, input_format_json_case_insensitive_column_matching, false, "Ignore case when matching JSON keys with CH columns", 0) \
M(UInt64, input_format_json_max_depth, 1000, "Maximum depth of a field in JSON. This is not a strict limit, it does not have to be applied precisely.", 0) \
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \

View File

@ -75,6 +75,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
},
{"24.8",
{
{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."},
{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"},
{"allow_archive_path_syntax", true, true, "Added new setting to allow disabling archive path syntax."},
}

View File

@ -123,6 +123,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.import_nested_json = settings.input_format_import_nested_json;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
format_settings.json.max_depth = settings.input_format_json_max_depth;
format_settings.json.array_of_rows = settings.output_format_json_array_of_rows;
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
format_settings.json.write_named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects;

View File

@ -205,6 +205,7 @@ struct FormatSettings
struct JSON
{
size_t max_depth = 1000;
bool array_of_rows = false;
bool quote_64bit_integers = true;
bool quote_64bit_floats = false;

View File

@ -3,13 +3,13 @@
#include <Common/PODArray.h>
#include <Common/StringUtils.h>
#include <Common/memcpySmall.h>
#include <Common/checkStackSize.h>
#include <Formats/FormatSettings.h>
#include <IO/WriteBufferFromString.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/readFloatText.h>
#include <IO/Operators.h>
#include <base/find_symbols.h>
#include <cstdlib>
#include <bit>
@ -39,6 +39,7 @@ namespace ErrorCodes
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int TOO_DEEP_RECURSION;
}
template <size_t num_bytes, typename IteratorSrc, typename IteratorDst>
@ -1494,10 +1495,20 @@ template bool readDateTimeTextFallback<bool, true>(time_t &, ReadBuffer &, const
template <typename ReturnType>
ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings)
ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings, size_t current_depth)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
if (unlikely(current_depth > settings.max_depth))
{
if constexpr (throw_exception)
throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "JSON is too deep for key '{}'", name_of_field.toString());
return ReturnType(false);
}
if (unlikely(current_depth > 0 && current_depth % 1024 == 0))
checkStackSize();
if (buf.eof())
{
if constexpr (throw_exception)
@ -1560,8 +1571,8 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo
while (true)
{
if constexpr (throw_exception)
skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings);
else if (!skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings))
skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings, current_depth + 1);
else if (!skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings, current_depth + 1))
return ReturnType(false);
skipWhitespaceIfAny(buf);
@ -1619,8 +1630,8 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo
skipWhitespaceIfAny(buf);
if constexpr (throw_exception)
skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings);
else if (!skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings))
skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings, current_depth + 1);
else if (!skipJSONFieldImpl<ReturnType>(buf, name_of_field, settings, current_depth + 1))
return ReturnType(false);
skipWhitespaceIfAny(buf);
@ -1659,12 +1670,12 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo
void skipJSONField(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings)
{
skipJSONFieldImpl<void>(buf, name_of_field, settings);
skipJSONFieldImpl<void>(buf, name_of_field, settings, 0);
}
bool trySkipJSONField(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings)
{
return skipJSONFieldImpl<bool>(buf, name_of_field, settings);
return skipJSONFieldImpl<bool>(buf, name_of_field, settings, 0);
}

View File

@ -113,13 +113,13 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
return res;
}
class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitor<ReplaceTableNodeToDummyVisitor, true>
class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitorWithContext<ReplaceTableNodeToDummyVisitor>
{
public:
using Base = InDepthQueryTreeVisitor<ReplaceTableNodeToDummyVisitor, true>;
using Base = InDepthQueryTreeVisitorWithContext<ReplaceTableNodeToDummyVisitor>;
using Base::Base;
void visitImpl(const QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
auto * table_node = node->as<TableNode>();
auto * table_function_node = node->as<TableFunctionNode>();
@ -134,21 +134,19 @@ public:
ColumnsDescription(storage_snapshot->getColumns(get_column_options)),
storage_snapshot);
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context);
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), getContext());
dummy_table_node->setAlias(node->getAlias());
replacement_map.emplace(node.get(), std::move(dummy_table_node));
}
}
ContextPtr context;
std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> replacement_map;
};
QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context)
QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const ContextPtr & context)
{
ReplaceTableNodeToDummyVisitor visitor;
visitor.context = context;
ReplaceTableNodeToDummyVisitor visitor(context);
visitor.visit(query);
return query->cloneAndReplace(visitor.replacement_map);

View File

@ -13,7 +13,7 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
struct SelectQueryOptions;
/// Find a qury which can be executed with parallel replicas up to WithMergableStage.
/// Find a query which can be executed with parallel replicas up to WithMergableStage.
/// Returned query will always contain some (>1) subqueries, possibly with joins.
const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options);

View File

@ -0,0 +1,5 @@
-- The default limit works.
SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA }
-- Even if we relax the limit, it is also safe.
SET input_format_json_max_depth = 100000;
SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA }

View File

@ -0,0 +1,15 @@
create table t (number UInt64) engine MergeTree order by number;
SELECT 1
FROM
(
SELECT number IN (
SELECT number
FROM view(
SELECT number
FROM numbers(1)
)
)
FROM t
)
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; -- { serverError CLUSTER_DOESNT_EXIST }