mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge pull request #67324 from ClickHouse/fix-json-recursion
Fix stack overflow in too deep JSON
This commit is contained in:
commit
06f83022da
@ -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) \
|
||||
|
@ -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."},
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
0
tests/queries/0_stateless/03213_deep_json.reference
Normal file
0
tests/queries/0_stateless/03213_deep_json.reference
Normal file
5
tests/queries/0_stateless/03213_deep_json.sql
Normal file
5
tests/queries/0_stateless/03213_deep_json.sql
Normal 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 }
|
@ -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 }
|
Loading…
Reference in New Issue
Block a user