merge master

This commit is contained in:
taiyang-li 2024-08-04 22:05:01 +08:00
commit d0cb299b22
32 changed files with 209 additions and 94 deletions

View File

@ -212,6 +212,10 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--shared-catalog')
fi
if [[ "$USE_DISTRIBUTED_CACHE" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--distributed-cache')
fi
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
# Too many tests fail for DatabaseReplicated in parallel.

View File

@ -43,7 +43,7 @@ Result:
## mapFromArrays
Creates a map from an array of keys and an array of values.
Creates a map from an array or map of keys and an array or map of values.
The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`.
For example, instead of writing
@ -62,8 +62,8 @@ Alias: `MAP_FROM_ARRAYS(keys, values)`
**Arguments**
- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type.
- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md).
- `keys` — Array or map of keys to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). If `keys` is an array, we accept `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as its type as long as it doesn't contain NULL value.
- `values` - Array or map of values to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md).
**Returned value**
@ -99,6 +99,18 @@ Result:
└───────────────────────────────────────────────────────┘
```
```sql
SELECT mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])
```
Result:
```
┌─mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])─┐
│ {('a',1):1,('b',2):2,('c',3):3} │
└───────────────────────────────────────────────────────┘
```
## extractKeyValuePairs
Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md).

View File

@ -19,6 +19,10 @@ TaskTracker::TaskTracker(ThreadPoolCallbackRunnerUnsafe<void> scheduler_, size_t
TaskTracker::~TaskTracker()
{
/// Tasks should be waited outside of dtor.
/// Important for WriteBufferFromS3/AzureBlobStorage, where TaskTracker is currently used.
chassert(finished_futures.empty() && futures.empty());
safeWaitAll();
}
@ -170,4 +174,3 @@ bool TaskTracker::isAsync() const
}
}

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

@ -1,9 +1,9 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnArray.h>
#include <base/arithmeticOverflow.h>
namespace DB
@ -15,7 +15,8 @@ namespace ErrorCodes
extern const int TOO_LARGE_ARRAY_SIZE;
}
/// Reasonable threshold.
/// Reasonable thresholds.
static constexpr Int64 max_array_size_in_columns_bytes = 1000000000;
static constexpr size_t max_arrays_size_in_columns = 1000000000;
@ -63,12 +64,19 @@ public:
auto array_size = col_num->getInt(i);
if (unlikely(array_size < 0))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size cannot be negative: while executing function {}", getName());
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} cannot be negative: while executing function {}", array_size, getName());
Int64 estimated_size = 0;
if (unlikely(common::mulOverflow(array_size, col_value->byteSize(), estimated_size)))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName());
if (unlikely(estimated_size > max_array_size_in_columns_bytes))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName());
offset += array_size;
if (unlikely(offset > max_arrays_size_in_columns))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size while executing function {}", getName());
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size {} (will generate at least {} elements) while executing function {}", array_size, offset, getName());
offsets.push_back(offset);
}

View File

@ -1,5 +1,9 @@
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
@ -11,7 +15,6 @@
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <Common/HashTable/HashSet.h>
#include <Core/Settings.h>
namespace DB
@ -22,6 +25,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
}
namespace
@ -139,7 +143,7 @@ private:
FunctionOverloadResolverPtr function_map_from_arrays;
};
/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays
/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays or maps
class FunctionMapFromArrays : public IFunction
{
public:
@ -163,21 +167,28 @@ public:
getName(),
arguments.size());
/// The first argument should always be Array.
/// Because key type can not be nested type of Map, which is Tuple
DataTypePtr key_type;
if (const auto * keys_type = checkAndGetDataType<DataTypeArray>(arguments[0].get()))
key_type = keys_type->getNestedType();
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName());
auto get_nested_type = [&](const DataTypePtr & type)
{
DataTypePtr nested;
if (const auto * type_as_array = checkAndGetDataType<DataTypeArray>(type.get()))
nested = type_as_array->getNestedType();
else if (const auto * type_as_map = checkAndGetDataType<DataTypeMap>(type.get()))
nested = std::make_shared<DataTypeTuple>(type_as_map->getKeyValueTypes());
else
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Arguments of function {} must be Array or Map, but {} is given",
getName(),
type->getName());
DataTypePtr value_type;
if (const auto * value_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get()))
value_type = value_array_type->getNestedType();
else if (const auto * value_map_type = checkAndGetDataType<DataTypeMap>(arguments[1].get()))
value_type = std::make_shared<DataTypeTuple>(value_map_type->getKeyValueTypes());
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName());
return nested;
};
auto key_type = get_nested_type(arguments[0]);
auto value_type = get_nested_type(arguments[1]);
/// We accept Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as key types as long as the actual array doesn't contain NULL value(this is checked in executeImpl).
key_type = removeNullableOrLowCardinalityNullable(key_type);
DataTypes key_value_types{key_type, value_type};
return std::make_shared<DataTypeMap>(key_value_types);
@ -186,44 +197,59 @@ public:
ColumnPtr executeImpl(
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override
{
bool is_keys_const = isColumnConst(*arguments[0].column);
ColumnPtr holder_keys;
const ColumnArray * col_keys;
if (is_keys_const)
auto get_array_column = [&](const ColumnPtr & column) -> std::pair<const ColumnArray *, ColumnPtr>
{
holder_keys = arguments[0].column->convertToFullColumnIfConst();
col_keys = checkAndGetColumn<ColumnArray>(holder_keys.get());
}
else
bool is_const = isColumnConst(*column);
ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column;
const ColumnArray * col_res = nullptr;
if (const auto * col_array = checkAndGetColumn<ColumnArray>(holder.get()))
col_res = col_array;
else if (const auto * col_map = checkAndGetColumn<ColumnMap>(holder.get()))
col_res = &col_map->getNestedColumn();
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Argument columns of function {} must be Array or Map, but {} is given",
getName(),
holder->getName());
return {col_res, holder};
};
auto [col_keys, key_holder] = get_array_column(arguments[0].column);
auto [col_values, values_holder] = get_array_column(arguments[1].column);
/// Nullable(T) or LowCardinality(Nullable(T)) are okay as nested key types but actual NULL values are not okay.
ColumnPtr data_keys = col_keys->getDataPtr();
if (isColumnNullableOrLowCardinalityNullable(*data_keys))
{
col_keys = checkAndGetColumn<ColumnArray>(arguments[0].column.get());
const NullMap * null_map = nullptr;
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(data_keys.get()))
{
null_map = &nullable->getNullMapData();
data_keys = nullable->getNestedColumnPtr();
}
else if (const auto * low_cardinality = checkAndGetColumn<ColumnLowCardinality>(data_keys.get()))
{
if (const auto * nullable_dict = checkAndGetColumn<ColumnNullable>(low_cardinality->getDictionaryPtr().get()))
{
null_map = &nullable_dict->getNullMapData();
data_keys = ColumnLowCardinality::create(nullable_dict->getNestedColumnPtr(), low_cardinality->getIndexesPtr());
}
}
if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size()))
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "The nested column of first argument in function {} must not contain NULLs", getName());
}
if (!col_keys)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName());
bool is_values_const = isColumnConst(*arguments[1].column);
ColumnPtr holder_values;
if (is_values_const)
holder_values = arguments[1].column->convertToFullColumnIfConst();
else
holder_values = arguments[1].column;
const ColumnArray * col_values;
if (const auto * col_values_array = checkAndGetColumn<ColumnArray>(holder_values.get()))
col_values = col_values_array;
else if (const auto * col_values_map = checkAndGetColumn<ColumnMap>(holder_values.get()))
col_values = &col_values_map->getNestedColumn();
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName());
if (!col_keys->hasEqualOffsets(*col_values))
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName());
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName());
const auto & data_keys = col_keys->getDataPtr();
const auto & data_values = col_values->getDataPtr();
const auto & offsets = col_keys->getOffsetsPtr();
auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets);
auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets);
return ColumnMap::create(nested_column);
}
};
@ -234,10 +260,7 @@ public:
static constexpr auto name = "mapUpdate";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapUpdate>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
@ -246,9 +269,11 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 2",
getName(), arguments.size());
getName(),
arguments.size());
const auto * left = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
const auto * right = checkAndGetDataType<DataTypeMap>(arguments[1].type.get());
@ -364,7 +389,6 @@ public:
return ColumnMap::create(nested_column);
}
};
}
REGISTER_FUNCTION(Map)

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

@ -277,12 +277,10 @@ WriteBufferFromS3::~WriteBufferFromS3()
"The file might not be written to S3. "
"{}.",
getVerboseLogDetails());
return;
}
/// That destructor could be call with finalized=false in case of exceptions
if (!finalized && !canceled)
else if (!finalized)
{
/// That destructor could be call with finalized=false in case of exceptions
LOG_INFO(
log,
"WriteBufferFromS3 is not finalized in destructor. "
@ -291,9 +289,10 @@ WriteBufferFromS3::~WriteBufferFromS3()
getVerboseLogDetails());
}
/// Wait for all tasks, because they contain reference to this write buffer.
task_tracker->safeWaitAll();
if (!multipart_upload_id.empty() && !multipart_upload_finished)
if (!canceled && !multipart_upload_id.empty() && !multipart_upload_finished)
{
LOG_WARNING(log, "WriteBufferFromS3 was neither finished nor aborted, try to abort upload in destructor. {}.", getVerboseLogDetails());
tryToAbortMultipartUpload();

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

@ -681,6 +681,7 @@ class FailureReason(enum.Enum):
BUILD = "not running for current build"
NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas"
SHARED_MERGE_TREE = "no-shared-merge-tree"
DISTRIBUTED_CACHE = "distributed-cache"
# UNKNOWN reasons
NO_REFERENCE = "no reference file"
@ -1191,6 +1192,9 @@ class TestCase:
elif tags and ("no-replicated-database" in tags) and args.replicated_database:
return FailureReason.REPLICATED_DB
elif tags and ("no-distributed-cache" in tags) and args.distributed_cache:
return FailureReason.DISTRIBUTED_CACHE
elif (
tags
and ("atomic-database" in tags)
@ -3203,6 +3207,12 @@ def parse_args():
default=False,
help="Run tests over s3 storage",
)
parser.add_argument(
"--distributed-cache",
action="store_true",
default=False,
help="Run tests with enabled distributed cache",
)
parser.add_argument(
"--azure-blob-storage",
action="store_true",

View File

@ -1,4 +1,4 @@
-- Tags: no-tsan
-- Tags: long, no-tsan, no-distributed-cache
-- Tag no-tsan: Too long for TSan
set enable_filesystem_cache=0;

View File

@ -1,4 +1,4 @@
-- Tags: no-random-merge-tree-settings, no-random-settings, no-tsan, no-debug, no-object-storage, long
-- Tags: long, no-random-merge-tree-settings, no-random-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache
-- no-tsan: too slow
-- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher

View File

@ -52,3 +52,10 @@
{1:4,2:5}
{1:4,2:5}
{1:4,2:5}
{1:3,2:4}
{1:3,2:4}
{1:3,2:4} {(1,3):'a',(2,4):'b'}
{(1,'a'):'c',(2,'b'):'d'}
{(1,'a'):'c',(2,'b'):'d'}
{(1,'a'):'c',(2,'b'):'d'}
{(1,'a'):'c',(2,'b'):'d'}

View File

@ -67,12 +67,20 @@ select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_
select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE}
select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE}
select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError BAD_ARGUMENTS }
select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5));
select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2);
select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], [4, 5]);
select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, 5])) from numbers(2);
select mapFromArrays([1,2], [3,4]);
select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]);
select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']);
select mapFromArrays(map(1, 'a', 2, 'b'), array('c', 'd'));
select mapFromArrays(materialize(map(1, 'a', 2, 'b')), array('c', 'd'));
select mapFromArrays(map(1, 'a', 2, 'b'), materialize(array('c', 'd')));
select mapFromArrays(materialize(map(1, 'a', 2, 'b')), materialize(array('c', 'd')));

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-random-settings, no-distributed-cache
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,3 +1,5 @@
-- Tags: no-distributed-cache
SET min_bytes_to_use_direct_io='1Gi'; -- It does not work (fixme)
SET local_filesystem_read_method='pread'; -- ui_uring local_fs_method does not work here (fixme)

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-random-settings
# Tags: no-fasttest, no-random-settings, no-distributed-cache
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,4 +1,4 @@
-- Tags: long, no-random-merge-tree-settings
-- Tags: long, no-random-merge-tree-settings, no-distributed-cache
-- no-random-merge-tree-settings - times out in private
DROP TABLE IF EXISTS build;

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 }

View File

@ -0,0 +1 @@
10000000

View File

@ -0,0 +1,3 @@
SELECT arrayWithConstant(96142475, ['qMUF']); -- { serverError TOO_LARGE_ARRAY_SIZE }
SELECT arrayWithConstant(100000000, materialize([[[[[[[[[['Hello, world!']]]]]]]]]])); -- { serverError TOO_LARGE_ARRAY_SIZE }
SELECT length(arrayWithConstant(10000000, materialize([[[[[[[[[['Hello world']]]]]]]]]])));