Merge branch 'dev_nested_upadte' of git://github.com/hexiaoting/ClickHouse into nested-update

This commit is contained in:
Anton Popov 2021-04-16 14:41:19 +03:00
commit b14798af2a
5 changed files with 231 additions and 2 deletions

View File

@ -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);

View File

@ -0,0 +1,118 @@
#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)
{
ColumnWithTypeAndName 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 = typeid_cast<const 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);
}
}
}
auto res = ColumnUInt8::create(input_rows_count);
auto & vec_res = res->getData();
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
vec_res[row_num] = 1;
return res;
}
void registerFunctionValidateNestedArraySizes(FunctionFactory & factory)
{
factory.registerFunction<FunctionValidateNestedArraySizes>();
}
}

View File

@ -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,34 @@ static void validateUpdateColumns(
}
}
std::pair<bool, std::vector<ASTPtr>> getFullNestedSubColumnUpdatedExpr(
const String & column,
NamesAndTypesList & all_columns,
std::unordered_map<String, ASTPtr> & column_to_update_expression)
{
std::vector<ASTPtr> res;
auto source_name = Nested::splitName(column).first;
/// Check this nested subcolumn
for (const auto & it : all_columns)
{
auto split = Nested::splitName(it.name);
if (split.first == source_name && !split.second.empty())
{
if (column_to_update_expression.find(it.name) == column_to_update_expression.end())
{
/// Update partial nested subcolumns
return std::make_pair(false, res);
}
else
{
res.push_back(column_to_update_expression[it.name]);
}
}
}
return std::make_pair(true, res);
}
ASTPtr MutationsInterpreter::prepare(bool dry_run)
{
@ -398,7 +427,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)
{
@ -441,9 +470,39 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName());
const auto & update_expr = kv.second;
ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command);
/// And new check validateNestedArraySizes for Nested subcolumns
if (!Nested::splitName(column).second.empty())
{
std::shared_ptr<ASTFunction> function = nullptr;
auto nested_update_exprs = getFullNestedSubColumnUpdatedExpr(column, all_columns, command.column_to_update_expression);
if (!nested_update_exprs.first)
{
function = makeASTFunction("validateNestedArraySizes",
condition,
update_expr->clone(),
std::make_shared<ASTIdentifier>(column));
condition = makeASTFunction("and", condition, function);
}
else if (nested_update_exprs.second.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.second)
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),

View File

@ -0,0 +1,20 @@
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]
waiting default test_wide_nested mutation_2.txt UPDATE `info.id` = [100, 200] WHERE id = 1
waiting default test_wide_nested mutation_3.txt UPDATE `info.id` = [100, 200, 300], `info.age` = [10, 20, 30], `info.name` = [\'a\', \'b\', \'c\'] WHERE id = 2
waiting default test_wide_nested mutation_4.txt UPDATE `info.id` = [100, 200], `info.age` = [10, 20, 30], `info.name` = [\'a\', \'b\', \'c\'] WHERE id = 0
waiting default test_wide_nested mutation_5.txt UPDATE `info.id` = [100, 200, 300], `info.age` = [10, 20, 30] WHERE id = 1
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]

View File

@ -0,0 +1,30 @@
USE default;
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;
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 settings mutations_sync=1;
select * from test_wide_nested where id = 1;
alter table test_wide_nested update `info.id` =[100,200,300], `info.age`=[10,20,30], `info.name`=['a','b','c'] where id = 2 settings mutations_sync=1;
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 settings mutations_sync=1; -- { serverError 341 }
kill mutation where table = 'test_wide_nested';
alter table test_wide_nested update `info.id` =[100,200,300], `info.age`=[10,20,30] where id = 1 settings mutations_sync=1; -- { serverError 341 }
kill mutation where table = '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 settings mutations_sync=1;
select * from test_wide_nested;
alter table test_wide_nested update `info.id` =[100,200], `info.age`=[68,72] where id = 3 settings mutations_sync=1;
alter table test_wide_nested update `info.id` = `info.age` where id = 3 settings mutations_sync=1;
select * from test_wide_nested;
DROP TABLE test_wide_nested;