mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
commit
8f57141fad
@ -45,6 +45,7 @@ void registerFunctionsUnixTimestamp64(FunctionFactory & factory);
|
||||
void registerFunctionBitHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionsStringHash(FunctionFactory & factory);
|
||||
void registerFunctionValidateNestedArraySizes(FunctionFactory & factory);
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
void registerFunctionBayesAB(FunctionFactory &);
|
||||
#endif
|
||||
@ -103,6 +104,7 @@ void registerFunctions()
|
||||
registerFunctionBitHammingDistance(factory);
|
||||
registerFunctionTupleHammingDistance(factory);
|
||||
registerFunctionsStringHash(factory);
|
||||
registerFunctionValidateNestedArraySizes(factory);
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
registerFunctionBayesAB(factory);
|
||||
|
113
src/Functions/validateNestedArraySizes.cpp
Normal file
113
src/Functions/validateNestedArraySizes.cpp
Normal file
@ -0,0 +1,113 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/** Function validateNestedArraySizes is used to check the consistency of Nested DataType subcolumns's offsets when Update
|
||||
* Arguments: num > 2
|
||||
* The first argument is the condition of WHERE in UPDATE operation, only when this is true, we need to check
|
||||
* The rest arguments are the subcolumns of Nested DataType.
|
||||
*/
|
||||
class FunctionValidateNestedArraySizes : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "validateNestedArraySizes";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionValidateNestedArraySizes>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override;
|
||||
};
|
||||
|
||||
DataTypePtr FunctionValidateNestedArraySizes::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
{
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
if (num_args < 3)
|
||||
throw Exception(
|
||||
"Function " + getName() + " needs more than two arguments; passed " + toString(arguments.size()) + ".",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!WhichDataType(arguments[0]).isUInt8())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + " Must be UInt.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
for (size_t i = 1; i < num_args; ++i)
|
||||
if (!WhichDataType(arguments[i]).isArray())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[i]->getName() + " of " + toString(i) + " argument of function " + getName() + " Must be Array.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
ColumnPtr FunctionValidateNestedArraySizes::executeImpl(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const
|
||||
{
|
||||
const ColumnUInt8 * condition_column = typeid_cast<const ColumnUInt8 *>(arguments[0].column.get());
|
||||
|
||||
size_t args_num = arguments.size();
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (!condition_column->getData()[i])
|
||||
continue;
|
||||
|
||||
/// The condition is true, then check the row in subcolumns in Nested Type has the same array size
|
||||
size_t first_length = 0;
|
||||
size_t length = 0;
|
||||
for (size_t args_idx = 1; args_idx < args_num; ++args_idx)
|
||||
{
|
||||
const auto & current_arg = arguments[args_idx];
|
||||
const ColumnArray * current_column = nullptr;
|
||||
if (const auto * const_array = checkAndGetColumnConst<ColumnArray>(current_arg.column.get()))
|
||||
{
|
||||
current_column = checkAndGetColumn<ColumnArray>(&const_array->getDataColumn());
|
||||
length = current_column->getOffsets()[0];
|
||||
}
|
||||
else
|
||||
{
|
||||
current_column = checkAndGetColumn<ColumnArray>(current_arg.column.get());
|
||||
const auto & offsets = current_column->getOffsets();
|
||||
length = offsets[i] - offsets[i - 1];
|
||||
}
|
||||
|
||||
if (args_idx == 1)
|
||||
{
|
||||
first_length = length;
|
||||
}
|
||||
else if (first_length != length)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH,
|
||||
"Elements '{}' and '{}' of Nested data structure (Array columns) "
|
||||
"have different array sizes ({} and {} respectively) on row {}",
|
||||
arguments[1].name, arguments[args_idx].name, first_length, length, i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ColumnUInt8::create(input_rows_count, 1);
|
||||
}
|
||||
|
||||
void registerFunctionValidateNestedArraySizes(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionValidateNestedArraySizes>();
|
||||
}
|
||||
|
||||
}
|
@ -26,6 +26,7 @@
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -349,6 +350,35 @@ static void validateUpdateColumns(
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns ASTs of updated nested subcolumns, if all of subcolumns were updated.
|
||||
/// They are used to validate sizes of nested arrays.
|
||||
/// If some of subcolumns were updated and some weren't,
|
||||
/// it makes sense to validate only updated columns with their old versions,
|
||||
/// because their sizes couldn't change, since sizes of all nested subcolumns must be consistent.
|
||||
static std::optional<std::vector<ASTPtr>> getExpressionsOfUpdatedNestedSubcolumns(
|
||||
const String & column_name,
|
||||
const NamesAndTypesList & all_columns,
|
||||
const std::unordered_map<String, ASTPtr> & column_to_update_expression)
|
||||
{
|
||||
std::vector<ASTPtr> res;
|
||||
auto source_name = Nested::splitName(column_name).first;
|
||||
|
||||
/// Check this nested subcolumn
|
||||
for (const auto & column : all_columns)
|
||||
{
|
||||
auto split = Nested::splitName(column.name);
|
||||
if (isArray(column.type) && split.first == source_name && !split.second.empty())
|
||||
{
|
||||
auto it = column_to_update_expression.find(column.name);
|
||||
if (it == column_to_update_expression.end())
|
||||
return {};
|
||||
|
||||
res.push_back(it->second);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
{
|
||||
@ -398,7 +428,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
/// First, break a sequence of commands into stages.
|
||||
for (const auto & command : commands)
|
||||
for (auto & command : commands)
|
||||
{
|
||||
if (command.type == MutationCommand::DELETE)
|
||||
{
|
||||
@ -438,12 +468,43 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
///
|
||||
/// Outer CAST is added just in case if we don't trust the returning type of 'if'.
|
||||
|
||||
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName());
|
||||
const auto & type = columns_desc.getPhysical(column).type;
|
||||
auto type_literal = std::make_shared<ASTLiteral>(type->getName());
|
||||
|
||||
const auto & update_expr = kv.second;
|
||||
|
||||
ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command);
|
||||
|
||||
/// And new check validateNestedArraySizes for Nested subcolumns
|
||||
if (isArray(type) && !Nested::splitName(column).second.empty())
|
||||
{
|
||||
std::shared_ptr<ASTFunction> function = nullptr;
|
||||
|
||||
auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column, all_columns, command.column_to_update_expression);
|
||||
if (!nested_update_exprs)
|
||||
{
|
||||
function = makeASTFunction("validateNestedArraySizes",
|
||||
condition,
|
||||
update_expr->clone(),
|
||||
std::make_shared<ASTIdentifier>(column));
|
||||
condition = makeASTFunction("and", condition, function);
|
||||
}
|
||||
else if (nested_update_exprs->size() > 1)
|
||||
{
|
||||
function = std::make_shared<ASTFunction>();
|
||||
function->name = "validateNestedArraySizes";
|
||||
function->arguments = std::make_shared<ASTExpressionList>();
|
||||
function->children.push_back(function->arguments);
|
||||
function->arguments->children.push_back(condition);
|
||||
for (const auto & it : *nested_update_exprs)
|
||||
function->arguments->children.push_back(it->clone());
|
||||
condition = makeASTFunction("and", condition, function);
|
||||
}
|
||||
}
|
||||
|
||||
auto updated_column = makeASTFunction("CAST",
|
||||
makeASTFunction("if",
|
||||
getPartitionAndPredicateExpressionForMutationCommand(command),
|
||||
condition,
|
||||
makeASTFunction("CAST",
|
||||
update_expr->clone(),
|
||||
type_literal),
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -393,8 +394,9 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name,
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName());
|
||||
|
||||
auto disk = data_part->volume->getDisk();
|
||||
String mrk_path = fullPath(disk, part_path + name + marks_file_extension);
|
||||
String bin_path = fullPath(disk, part_path + name + DATA_FILE_EXTENSION);
|
||||
String escaped_name = escapeForFileName(name);
|
||||
String mrk_path = fullPath(disk, part_path + escaped_name + marks_file_extension);
|
||||
String bin_path = fullPath(disk, part_path + escaped_name + DATA_FILE_EXTENSION);
|
||||
DB::ReadBufferFromFile mrk_in(mrk_path);
|
||||
DB::CompressedReadBufferFromFile bin_in(bin_path, 0, 0, 0, nullptr);
|
||||
bool must_be_last = false;
|
||||
|
@ -0,0 +1,21 @@
|
||||
1 [100,200] ['aa','bb'] [1,2]
|
||||
0 [0,1] ['aa','bb'] [0,0]
|
||||
1 [100,200] ['aa','bb'] [1,2]
|
||||
2 [100,200,300] ['a','b','c'] [10,20,30]
|
||||
3 [3,4] ['aa','bb'] [3,6]
|
||||
4 [4,5] ['aa','bb'] [4,8]
|
||||
0 [0,1] ['aa','bb'] [0,0]
|
||||
1 [100,200] ['aa','bb'] [1,2]
|
||||
2 [100,200,300] ['a','b','c'] [100,200,300]
|
||||
3 [3,4] ['aa','bb'] [3,6]
|
||||
4 [4,5] ['aa','bb'] [4,8]
|
||||
0 [0,1] ['aa','bb'] [0,0]
|
||||
1 [100,200] ['aa','bb'] [1,2]
|
||||
2 [100,200,300] ['a','b','c'] [100,200,300]
|
||||
3 [68,72] ['aa','bb'] [68,72]
|
||||
4 [4,5] ['aa','bb'] [4,8]
|
||||
0 0 aa 0
|
||||
1 1 bb 2
|
||||
2 2 aa 4
|
||||
3 3 aa 6
|
||||
4 4 aa 8
|
@ -0,0 +1,70 @@
|
||||
DROP TABLE IF EXISTS test_wide_nested;
|
||||
|
||||
CREATE TABLE test_wide_nested
|
||||
(
|
||||
`id` Int,
|
||||
`info.id` Array(Int),
|
||||
`info.name` Array(String),
|
||||
`info.age` Array(Int)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
set mutations_sync = 1;
|
||||
|
||||
INSERT INTO test_wide_nested SELECT number, [number,number + 1], ['aa','bb'], [number,number * 2] FROM numbers(5);
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200] where id = 1;
|
||||
select * from test_wide_nested where id = 1 order by id;
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = [10,20,30], `info.name` = ['a','b','c'] where id = 2;
|
||||
select * from test_wide_nested;
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = `info.id`, `info.name` = ['a','b','c'] where id = 2;
|
||||
select * from test_wide_nested;
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200], `info.age`=[68,72] where id = 3;
|
||||
alter table test_wide_nested update `info.id` = `info.age` where id = 3;
|
||||
select * from test_wide_nested;
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200], `info.age` = [10,20,30], `info.name` = ['a','b','c'] where id = 0; -- { serverError 341 }
|
||||
|
||||
-- Recreate table, because KILL MUTATION is not suitable for parallel tests execution.
|
||||
DROP TABLE test_wide_nested;
|
||||
|
||||
CREATE TABLE test_wide_nested
|
||||
(
|
||||
`id` Int,
|
||||
`info.id` Array(Int),
|
||||
`info.name` Array(String),
|
||||
`info.age` Array(Int)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO test_wide_nested SELECT number, [number,number + 1], ['aa','bb'], [number,number * 2] FROM numbers(5);
|
||||
|
||||
alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = [10,20,30] where id = 1; -- { serverError 341 }
|
||||
|
||||
DROP TABLE test_wide_nested;
|
||||
|
||||
DROP TABLE IF EXISTS test_wide_not_nested;
|
||||
|
||||
CREATE TABLE test_wide_not_nested
|
||||
(
|
||||
`id` Int,
|
||||
`info.id` Int,
|
||||
`info.name` String,
|
||||
`info.age` Int
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO test_wide_not_nested SELECT number, number, 'aa', number * 2 FROM numbers(5);
|
||||
ALTER TABLE test_wide_not_nested UPDATE `info.name` = 'bb' WHERE id = 1;
|
||||
SELECT * FROM test_wide_not_nested ORDER BY id;
|
||||
|
||||
DROP TABLE test_wide_not_nested;
|
Loading…
Reference in New Issue
Block a user