mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Merge branch 'master' into dictionaries_ddl_interpreter
This commit is contained in:
commit
3dfe961291
@ -1540,6 +1540,9 @@ public:
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
{
|
||||
return static_cast<bool>(monotonicity_for_range);
|
||||
|
@ -26,6 +26,9 @@ public:
|
||||
|
||||
String getName() const override { return "FunctionExpression"; }
|
||||
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
@ -110,6 +113,9 @@ public:
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return captured_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
|
@ -53,9 +53,11 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
|
@ -198,9 +198,9 @@ public:
|
||||
* Example: now(). Another example: functions that work with periodically updated dictionaries.
|
||||
*/
|
||||
|
||||
virtual bool isDeterministic() const { return true; }
|
||||
virtual bool isDeterministic() const = 0;
|
||||
|
||||
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||
virtual bool isDeterministicInScopeOfQuery() const = 0;
|
||||
|
||||
/** Lets you know if the function is monotonic in a range of values.
|
||||
* This is used to work with the index in a sorted chunk of data.
|
||||
@ -240,11 +240,16 @@ public:
|
||||
/// Get the main function name.
|
||||
virtual String getName() const = 0;
|
||||
|
||||
/// See the comment for the same method in IFunctionBase
|
||||
virtual bool isDeterministic() const = 0;
|
||||
|
||||
virtual bool isDeterministicInScopeOfQuery() const = 0;
|
||||
|
||||
/// Override and return true if function needs to depend on the state of the data.
|
||||
virtual bool isStateful() const { return false; }
|
||||
virtual bool isStateful() const = 0;
|
||||
|
||||
/// Override and return true if function could take different number of arguments.
|
||||
virtual bool isVariadic() const { return false; }
|
||||
virtual bool isVariadic() const = 0;
|
||||
|
||||
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
|
||||
virtual size_t getNumberOfArguments() const = 0;
|
||||
@ -277,6 +282,11 @@ public:
|
||||
return buildImpl(arguments, getReturnType(arguments));
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
bool isStateful() const override { return false; }
|
||||
bool isVariadic() const override { return false; }
|
||||
|
||||
/// Default implementation. Will check only in non-variadic case.
|
||||
void checkNumberOfArguments(size_t number_of_arguments) const override;
|
||||
|
||||
@ -357,6 +367,8 @@ public:
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
|
||||
bool canBeExecutedOnDefaultArguments() const override { return true; }
|
||||
bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); }
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
using PreparedFunctionImpl::execute;
|
||||
using PreparedFunctionImpl::executeImplDryRun;
|
||||
@ -506,6 +518,9 @@ public:
|
||||
return function->checkNumberOfArguments(number_of_arguments);
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return function->isDeterministic(); }
|
||||
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
|
||||
|
||||
String getName() const override { return function->getName(); }
|
||||
bool isStateful() const override { return function->isStateful(); }
|
||||
bool isVariadic() const override { return function->isVariadic(); }
|
||||
|
@ -53,6 +53,7 @@ public:
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
private:
|
||||
time_t time_value;
|
||||
@ -65,8 +66,10 @@ public:
|
||||
static constexpr auto name = "now";
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
static FunctionBuilderPtr create(const Context &) { return std::make_shared<FunctionBuilderNow>(); }
|
||||
|
||||
protected:
|
||||
|
@ -50,6 +50,7 @@ public:
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
private:
|
||||
ToType value;
|
||||
@ -64,6 +65,8 @@ public:
|
||||
static constexpr auto name = Name::name;
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
|
@ -141,6 +141,7 @@ public:
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override
|
||||
{
|
||||
return false;
|
||||
|
@ -38,6 +38,9 @@ public:
|
||||
static constexpr auto name = "toTypeName";
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return true; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
|
@ -52,6 +52,7 @@ public:
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
private:
|
||||
DayNum day_value;
|
||||
@ -64,6 +65,9 @@ public:
|
||||
static constexpr auto name = "today";
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
static FunctionBuilderPtr create(const Context &) { return std::make_shared<FunctionBuilderToday>(); }
|
||||
|
@ -52,6 +52,7 @@ public:
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
private:
|
||||
DayNum day_value;
|
||||
@ -64,6 +65,9 @@ public:
|
||||
static constexpr auto name = "yesterday";
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
static FunctionBuilderPtr create(const Context &) { return std::make_shared<FunctionBuilderYesterday>(); }
|
||||
|
@ -1,6 +1,11 @@
|
||||
#include "MutationsInterpreter.h"
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DataStreams/FilterBlockInputStream.h>
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
@ -14,7 +19,6 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "MutationsInterpreter.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -27,6 +31,67 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_UPDATE_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
struct FirstNonDeterministicFuncData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
||||
explicit FirstNonDeterministicFuncData(const Context & context_)
|
||||
: context{context_}
|
||||
{}
|
||||
|
||||
const Context & context;
|
||||
std::optional<String> nondeterministic_function_name;
|
||||
|
||||
void visit(ASTFunction & function, ASTPtr &)
|
||||
{
|
||||
if (nondeterministic_function_name)
|
||||
return;
|
||||
|
||||
const auto func = FunctionFactory::instance().get(function.name, context);
|
||||
if (!func->isDeterministic())
|
||||
nondeterministic_function_name = func->getName();
|
||||
}
|
||||
};
|
||||
|
||||
using FirstNonDeterministicFuncFinder =
|
||||
InDepthNodeVisitor<OneTypeMatcher<FirstNonDeterministicFuncData>, true>;
|
||||
|
||||
std::optional<String> findFirstNonDeterministicFuncName(const MutationCommand & command, const Context & context)
|
||||
{
|
||||
FirstNonDeterministicFuncData finder_data(context);
|
||||
|
||||
switch (command.type)
|
||||
{
|
||||
case MutationCommand::UPDATE:
|
||||
{
|
||||
auto update_assignments_ast = command.ast->as<const ASTAlterCommand &>().update_assignments->clone();
|
||||
FirstNonDeterministicFuncFinder(finder_data).visit(update_assignments_ast);
|
||||
|
||||
if (finder_data.nondeterministic_function_name)
|
||||
return finder_data.nondeterministic_function_name;
|
||||
|
||||
[[fallthrough]];
|
||||
}
|
||||
|
||||
case MutationCommand::DELETE:
|
||||
{
|
||||
auto predicate_ast = command.predicate->clone();
|
||||
FirstNonDeterministicFuncFinder(finder_data).visit(predicate_ast);
|
||||
|
||||
return finder_data.nondeterministic_function_name;
|
||||
}
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
bool MutationsInterpreter::isStorageTouchedByMutations() const
|
||||
{
|
||||
if (commands.empty())
|
||||
@ -440,6 +505,21 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve
|
||||
|
||||
void MutationsInterpreter::validate(TableStructureReadLockHolder &)
|
||||
{
|
||||
/// For Replicated* storages mutations cannot employ non-deterministic functions
|
||||
/// because that produces inconsistencies between replicas
|
||||
if (startsWith(storage->getName(), "Replicated"))
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
const auto nondeterministic_func_name = findFirstNonDeterministicFuncName(command, context);
|
||||
if (nondeterministic_func_name)
|
||||
throw Exception(
|
||||
"ALTER UPDATE/ALTER DELETE statements must use only deterministic functions! "
|
||||
"Function '" + *nondeterministic_func_name + "' is non-deterministic",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
const auto & select_query = prepare(/* dry_run = */ true);
|
||||
InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ true).ignoreLimits()};
|
||||
/// Do not use getSampleBlock in order to check the whole pipeline.
|
||||
|
@ -60,11 +60,11 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
|
||||
|
||||
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context)
|
||||
{
|
||||
/// Branch with string in query.
|
||||
/// If it's already a literal.
|
||||
if (node->as<ASTLiteral>())
|
||||
return node;
|
||||
|
||||
/// Branch with TableFunction in query.
|
||||
/// Skip table functions.
|
||||
if (const auto * table_func_ptr = node->as<ASTFunction>())
|
||||
if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name))
|
||||
return node;
|
||||
|
@ -20,6 +20,7 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
/** Evaluate constant expression and its type.
|
||||
* Used in rare cases - for elements of set for IN, for data to INSERT.
|
||||
* Throws exception if it's not a constant expression.
|
||||
* Quite suboptimal.
|
||||
*/
|
||||
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const Context & context);
|
||||
|
@ -120,6 +120,10 @@ static void logQuery(const String & query, const Context & context, bool interna
|
||||
/// Call this inside catch block.
|
||||
static void setExceptionStackTrace(QueryLogElement & elem)
|
||||
{
|
||||
/// Disable memory tracker for stack trace.
|
||||
/// Because if exception is "Memory limit (for query) exceed", then we probably can't allocate another one string.
|
||||
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
|
||||
|
||||
try
|
||||
{
|
||||
throw;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Storages/StorageMemory.h>
|
||||
@ -17,7 +18,13 @@ using namespace DB;
|
||||
struct State
|
||||
{
|
||||
Context context{Context::createGlobal()};
|
||||
NamesAndTypesList columns{{"column", std::make_shared<DataTypeUInt8>()}};
|
||||
NamesAndTypesList columns{
|
||||
{"column", std::make_shared<DataTypeUInt8>()},
|
||||
{"apply_id", std::make_shared<DataTypeUInt64>()},
|
||||
{"apply_type", std::make_shared<DataTypeUInt8>()},
|
||||
{"apply_status", std::make_shared<DataTypeUInt8>()},
|
||||
{"create_time", std::make_shared<DataTypeDateTime>()},
|
||||
};
|
||||
|
||||
State()
|
||||
{
|
||||
@ -85,5 +92,11 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries)
|
||||
check("SELECT column FROM test.table WHERE toString(column) = '42' AND left(column, 10) = RIGHT(column, 10) AND column = 42",
|
||||
"SELECT \"column\" FROM \"test\".\"table\" WHERE (\"column\" = 42)",
|
||||
state().context, state().columns);
|
||||
|
||||
}
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Issue7245)
|
||||
{
|
||||
check("select apply_id from test.table where apply_type = 2 and create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) and apply_status in (3,4)",
|
||||
"SELECT \"apply_id\", \"apply_type\", \"apply_status\", \"create_time\" FROM \"test\".\"table\" WHERE (\"apply_type\" = 2) AND (\"create_time\" > '2018-12-25 01:02:03') AND (\"apply_status\" IN (3, 4))",
|
||||
state().context, state().columns);
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <sstream>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -8,6 +9,8 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
|
||||
@ -20,31 +23,64 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static void replaceConstFunction(IAST & node, const Context & context, const NamesAndTypesList & all_columns)
|
||||
namespace
|
||||
{
|
||||
for (size_t i = 0; i < node.children.size(); ++i)
|
||||
{
|
||||
auto child = node.children[i];
|
||||
if (auto * exp_list = child->as<ASTExpressionList>())
|
||||
replaceConstFunction(*exp_list, context, all_columns);
|
||||
|
||||
if (auto * function = child->as<ASTFunction>())
|
||||
class ReplacingConstantExpressionsMatcher
|
||||
{
|
||||
public:
|
||||
using Data = Block;
|
||||
|
||||
static bool needChildVisit(ASTPtr &, const ASTPtr &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
static void visit(ASTPtr & node, Block & block_with_constants)
|
||||
{
|
||||
if (!node->as<ASTFunction>())
|
||||
return;
|
||||
|
||||
std::string name = node->getColumnName();
|
||||
if (block_with_constants.has(name))
|
||||
{
|
||||
NamesAndTypesList source_columns = all_columns;
|
||||
ASTPtr query = function->ptr();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, source_columns);
|
||||
auto result_block = KeyCondition::getBlockWithConstants(query, syntax_result, context);
|
||||
if (!result_block.has(child->getColumnName()))
|
||||
auto result = block_with_constants.getByName(name);
|
||||
if (!isColumnConst(*result.column))
|
||||
return;
|
||||
|
||||
auto result_column = result_block.getByName(child->getColumnName()).column;
|
||||
if (result.column->isNullAt(0))
|
||||
{
|
||||
node = std::make_shared<ASTLiteral>(Field());
|
||||
}
|
||||
else if (isNumber(result.type))
|
||||
{
|
||||
node = std::make_shared<ASTLiteral>(assert_cast<const ColumnConst &>(*result.column).getField());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Everything except numbers is put as string literal. This is important for Date, DateTime, UUID.
|
||||
|
||||
node.children[i] = std::make_shared<ASTLiteral>((*result_column)[0]);
|
||||
const IColumn & inner_column = assert_cast<const ColumnConst &>(*result.column).getDataColumn();
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
result.type->serializeAsText(inner_column, 0, out, FormatSettings());
|
||||
node = std::make_shared<ASTLiteral>(out.str());
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
void replaceConstantExpressions(ASTPtr & node, const Context & context, const NamesAndTypesList & all_columns)
|
||||
{
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(node, all_columns);
|
||||
Block block_with_constants = KeyCondition::getBlockWithConstants(node, syntax_result, context);
|
||||
|
||||
InDepthNodeVisitor<ReplacingConstantExpressionsMatcher, true> visitor(block_with_constants);
|
||||
visitor.visit(node);
|
||||
}
|
||||
|
||||
static bool isCompatible(const IAST & node)
|
||||
|
||||
bool isCompatible(const IAST & node)
|
||||
{
|
||||
if (const auto * function = node.as<ASTFunction>())
|
||||
{
|
||||
@ -99,6 +135,8 @@ static bool isCompatible(const IAST & node)
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
String transformQueryForExternalDatabase(
|
||||
const IAST & query,
|
||||
@ -131,7 +169,8 @@ String transformQueryForExternalDatabase(
|
||||
ASTPtr original_where = clone_query->as<ASTSelectQuery &>().where();
|
||||
if (original_where)
|
||||
{
|
||||
replaceConstFunction(*original_where, context, available_columns);
|
||||
replaceConstantExpressions(original_where, context, available_columns);
|
||||
|
||||
if (isCompatible(*original_where))
|
||||
{
|
||||
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where));
|
||||
|
@ -214,7 +214,7 @@ def run_tests_array(all_tests_with_params):
|
||||
result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE)
|
||||
|
||||
if result_is_different:
|
||||
diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0]
|
||||
diff = Popen(['diff', '-U', str(args.unified), reference_file, stdout_file], stdout = PIPE).communicate()[0]
|
||||
failures += 1
|
||||
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff))
|
||||
else:
|
||||
@ -456,6 +456,7 @@ if __name__ == '__main__':
|
||||
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')
|
||||
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
|
||||
parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel')
|
||||
parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context')
|
||||
|
||||
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
|
||||
parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests')
|
||||
|
@ -1,3 +1,4 @@
|
||||
uniqHLL12
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -50,6 +51,7 @@
|
||||
31 54151
|
||||
35 54328
|
||||
36 52997
|
||||
uniqHLL12 round(float)
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -102,6 +104,7 @@
|
||||
0.043 54620
|
||||
0.037 53394
|
||||
0.071 53951
|
||||
uniqHLL12 round(toFloat32())
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -154,6 +157,7 @@
|
||||
0.037 53394
|
||||
0.1 54138
|
||||
1 54571
|
||||
uniqHLL12 IPv4NumToString
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -206,7 +210,9 @@
|
||||
31 53574
|
||||
35 55022
|
||||
36 53961
|
||||
uniqHLL12 remote()
|
||||
1
|
||||
uniqCombined
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -259,6 +265,7 @@
|
||||
31 53948
|
||||
35 53931
|
||||
36 53982
|
||||
uniqCombined(12)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -311,6 +318,7 @@
|
||||
31 53763
|
||||
35 54635
|
||||
36 53155
|
||||
uniqCombined(17)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -363,6 +371,7 @@
|
||||
31 53948
|
||||
35 53931
|
||||
36 53982
|
||||
uniqCombined(20)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -415,6 +424,7 @@
|
||||
31 54054
|
||||
35 54054
|
||||
36 54054
|
||||
uniqCombined(round(float))
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -467,6 +477,7 @@
|
||||
0.043 54150
|
||||
0.037 54047
|
||||
0.071 53963
|
||||
uniqCombined(12)(round(float))
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -519,6 +530,7 @@
|
||||
0.043 53827
|
||||
0.037 53920
|
||||
0.071 53409
|
||||
uniqCombined(17)(round(float))
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -571,6 +583,7 @@
|
||||
0.043 54150
|
||||
0.037 54047
|
||||
0.071 53963
|
||||
uniqCombined(20)(round(float))
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -623,6 +636,7 @@
|
||||
0.043 54053
|
||||
0.037 54053
|
||||
0.071 54054
|
||||
uniqCombined(X)(round(toFloat32()))
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -675,6 +689,7 @@
|
||||
0.037 54047
|
||||
0.1 53853
|
||||
1 53901
|
||||
uniqCombined(12)(round(toFloat32()))
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -727,6 +742,7 @@
|
||||
0.037 53920
|
||||
0.1 53417
|
||||
1 54708
|
||||
uniqCombined(17)(round(toFloat32()))
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -779,6 +795,7 @@
|
||||
0.037 54047
|
||||
0.1 53853
|
||||
1 53901
|
||||
uniqCombined(20)(round(toFloat32()))
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -831,6 +848,7 @@
|
||||
0.037 54053
|
||||
0.1 54053
|
||||
1 54054
|
||||
uniqCombined(Z)(IPv4NumToString)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -883,6 +901,7 @@
|
||||
31 54074
|
||||
35 54153
|
||||
36 53999
|
||||
uniqCombined(12)(IPv4NumToString)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -897,24 +916,24 @@
|
||||
31 1
|
||||
35 1
|
||||
36 1
|
||||
0 162
|
||||
0 161
|
||||
1 162
|
||||
3 162
|
||||
6 162
|
||||
7 163
|
||||
9 163
|
||||
7 164
|
||||
9 162
|
||||
10 81
|
||||
11 163
|
||||
13 162
|
||||
14 162
|
||||
17 162
|
||||
19 162
|
||||
20 162
|
||||
21 162
|
||||
22 162
|
||||
11 160
|
||||
13 163
|
||||
14 161
|
||||
17 159
|
||||
19 165
|
||||
20 163
|
||||
21 163
|
||||
22 158
|
||||
26 162
|
||||
31 162
|
||||
35 162
|
||||
35 164
|
||||
36 162
|
||||
0 52613
|
||||
1 54468
|
||||
@ -935,6 +954,7 @@
|
||||
31 55200
|
||||
35 54808
|
||||
36 53051
|
||||
uniqCombined(17)(IPv4NumToString)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -987,6 +1007,7 @@
|
||||
31 54074
|
||||
35 54153
|
||||
36 53999
|
||||
uniqCombined(20)(IPv4NumToString)
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
@ -1020,25 +1041,26 @@
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54054
|
||||
1 54054
|
||||
3 54054
|
||||
6 54054
|
||||
7 54054
|
||||
9 54054
|
||||
0 54038
|
||||
1 54104
|
||||
3 54033
|
||||
6 54084
|
||||
7 54081
|
||||
9 54093
|
||||
10 27027
|
||||
11 54055
|
||||
13 54054
|
||||
14 54054
|
||||
17 54054
|
||||
19 54054
|
||||
20 54054
|
||||
21 54054
|
||||
22 54054
|
||||
26 54054
|
||||
31 54054
|
||||
35 54054
|
||||
36 54054
|
||||
11 54064
|
||||
13 54055
|
||||
14 54063
|
||||
17 54055
|
||||
19 53960
|
||||
20 54033
|
||||
21 53988
|
||||
22 54086
|
||||
26 54106
|
||||
31 54039
|
||||
35 54018
|
||||
36 54084
|
||||
uniqCombined remote()
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -1,93 +1,134 @@
|
||||
/* uniqHLL12 */
|
||||
-- uniqHLL12
|
||||
|
||||
SELECT 'uniqHLL12';
|
||||
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqHLL12 round(float)';
|
||||
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqHLL12 round(toFloat32())';
|
||||
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqHLL12 IPv4NumToString';
|
||||
|
||||
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqHLL12 remote()';
|
||||
|
||||
SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
/* uniqCombined */
|
||||
-- uniqCombined
|
||||
|
||||
SELECT 'uniqCombined';
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(12)';
|
||||
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(17)';
|
||||
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(20)';
|
||||
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(round(float))';
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(12)(round(float))';
|
||||
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(17)(round(float))';
|
||||
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(20)(round(float))';
|
||||
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(X)(round(toFloat32()))';
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(12)(round(toFloat32()))';
|
||||
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(17)(round(toFloat32()))';
|
||||
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(20)(round(toFloat32()))';
|
||||
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(Z)(IPv4NumToString)';
|
||||
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(12)(IPv4NumToString)';
|
||||
|
||||
SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(17)(IPv4NumToString)';
|
||||
|
||||
SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined(20)(IPv4NumToString)';
|
||||
|
||||
SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT 'uniqCombined remote()';
|
||||
|
||||
SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
SELECT uniqCombined(12)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
SELECT uniqCombined(17)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
SELECT uniqCombined(20)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
@ -0,0 +1,4 @@
|
||||
OK
|
||||
OK
|
||||
OK
|
||||
OK
|
@ -0,0 +1,43 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
|
||||
R1=table_1017_1
|
||||
R2=table_1017_2
|
||||
T1=table_1017_merge
|
||||
|
||||
${CLICKHOUSE_CLIENT} -n -q "
|
||||
DROP TABLE IF EXISTS $R1;
|
||||
DROP TABLE IF EXISTS $R2;
|
||||
|
||||
CREATE TABLE $R1 (x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1017', 'r1') ORDER BY x;
|
||||
CREATE TABLE $R2 (x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1017', 'r2') ORDER BY x;
|
||||
CREATE TABLE $T1 (x UInt32, y UInt32) ENGINE MergeTree() ORDER BY x;
|
||||
|
||||
INSERT INTO $R1 VALUES (0, 1)(1, 2)(2, 3)(3, 4);
|
||||
INSERT INTO $T1 VALUES (0, 1)(1, 2)(2, 3)(3, 4);
|
||||
"
|
||||
|
||||
# Check that in mutations of replicated tables predicates do not contain non-deterministic functions
|
||||
${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE ignore(rand())" 2>&1 \
|
||||
| fgrep -q "must use only deterministic functions" && echo 'OK' || echo 'FAIL'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 UPDATE y = y + rand() % 1 WHERE not ignore()" 2>&1 \
|
||||
| fgrep -q "must use only deterministic functions" && echo 'OK' || echo 'FAIL'
|
||||
|
||||
|
||||
# For regular tables we do not enforce deterministic functions
|
||||
${CLICKHOUSE_CLIENT} --query "ALTER TABLE $T1 DELETE WHERE rand() = 0" 2>&1 > /dev/null \
|
||||
&& echo 'OK' || echo 'FAIL'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "ALTER TABLE $T1 UPDATE y = y + rand() % 1 WHERE not ignore()" 2>&1 > /dev/null \
|
||||
&& echo 'OK' || echo 'FAIL'
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} -n -q "
|
||||
DROP TABLE IF EXISTS $R2;
|
||||
DROP TABLE IF EXISTS $R1;
|
||||
DROP TABLE IF EXISTS $T1;
|
||||
"
|
Loading…
Reference in New Issue
Block a user