mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
dbms: implement concat for arbitrary arity (> 0). [#METR-12888]
This commit is contained in:
parent
31af725ba1
commit
fe49e06e46
@ -12,6 +12,7 @@
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -968,24 +969,94 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
+ toString(arguments.size()) + ", should be at least 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(&*arguments[0]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(&*arguments[1]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
for (const auto arg_idx : ext::range(0, arguments.size()))
|
||||
{
|
||||
const auto arg = arguments[arg_idx].get();
|
||||
if (!typeid_cast<const DataTypeString *>(arg) &&
|
||||
!typeid_cast<const DataTypeFixedString *>(arg))
|
||||
throw Exception{
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
enum class instr_type : uint8_t
|
||||
{
|
||||
copy_string,
|
||||
copy_fixed_string,
|
||||
copy_const_string
|
||||
};
|
||||
|
||||
/// column pointer augmented with offset (current offset String/FixedString, unused for Const<String>)
|
||||
using column_uint_pair_t = std::pair<const IColumn *, IColumn::Offset_t>;
|
||||
/// instr_type is being stored to allow using static_cast safely
|
||||
using instr_t = std::pair<instr_type, column_uint_pair_t>;
|
||||
using instrs_t = std::vector<instr_t>;
|
||||
|
||||
/** calculate total length of resulting strings (without terminating nulls), determine whether all input
|
||||
* strings are constant, assemble instructions */
|
||||
instrs_t getInstructions(const Block & block, const ColumnNumbers & arguments, size_t & out_length, bool & out_const)
|
||||
{
|
||||
instrs_t result{};
|
||||
result.reserve(arguments.size());
|
||||
|
||||
out_length = 0;
|
||||
out_const = true;
|
||||
|
||||
for (const auto arg_pos : arguments)
|
||||
{
|
||||
const auto column = block.getByPosition(arg_pos).column.get();
|
||||
|
||||
if (const auto col = typeid_cast<const ColumnString *>(column))
|
||||
{
|
||||
/** ColumnString stores strings with terminating null character
|
||||
* which should not be copied, therefore the decrease of total size by
|
||||
* the number of terminating nulls */
|
||||
out_length += col->getChars().size() - col->getOffsets().size();
|
||||
out_const = false;
|
||||
|
||||
result.emplace_back(instr_type::copy_string, column_uint_pair_t{col, 0});
|
||||
}
|
||||
else if (const auto col = typeid_cast<const ColumnFixedString *>(column))
|
||||
{
|
||||
out_length += col->getChars().size();
|
||||
out_const = false;
|
||||
|
||||
result.emplace_back(instr_type::copy_fixed_string, column_uint_pair_t{col, 0});
|
||||
}
|
||||
else if (const auto col = typeid_cast<const ColumnConstString *>(column))
|
||||
{
|
||||
out_length += col->getData().size();
|
||||
out_const = out_const && true;
|
||||
|
||||
result.emplace_back(instr_type::copy_const_string, column_uint_pair_t{col, 0});
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
executeBinary(block, arguments, result);
|
||||
else
|
||||
executeNAry(block, arguments, result);
|
||||
}
|
||||
|
||||
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
const IColumn * c0 = &*block.getByPosition(arguments[0]).column;
|
||||
const IColumn * c1 = &*block.getByPosition(arguments[1]).column;
|
||||
@ -1059,6 +1130,81 @@ public:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
const auto size = block.rowsInFirstColumn();
|
||||
std::size_t result_length{};
|
||||
bool result_is_const{};
|
||||
auto instrs = getInstructions(block, arguments, result_length, result_is_const);
|
||||
|
||||
if (result_is_const)
|
||||
{
|
||||
const auto out = new ColumnConst<String>{size, ""};
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto & data = out->getData();
|
||||
data.reserve(result_length);
|
||||
|
||||
for (const auto & instr : instrs)
|
||||
data += static_cast<const ColumnConst<String> *>(instr.second.first)->getData();
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto out = new ColumnString{};
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto & out_data = out->getChars();
|
||||
out_data.resize(result_length + size);
|
||||
|
||||
auto & out_offsets = out->getOffsets();
|
||||
out_offsets.resize(size);
|
||||
|
||||
std::size_t out_offset{};
|
||||
|
||||
for (const auto row : ext::range(0, size))
|
||||
{
|
||||
for (auto & instr : instrs)
|
||||
{
|
||||
if (instr_type::copy_string == instr.first)
|
||||
{
|
||||
auto & in_offset = instr.second.second;
|
||||
const auto col = static_cast<const ColumnString *>(instr.second.first);
|
||||
const auto offset = col->getOffsets()[row];
|
||||
const auto length = offset - in_offset - 1;
|
||||
|
||||
memcpy(&out_data[out_offset], &col->getChars()[in_offset], length);
|
||||
out_offset += length;
|
||||
in_offset = offset;
|
||||
}
|
||||
else if (instr_type::copy_fixed_string == instr.first)
|
||||
{
|
||||
auto & in_offset = instr.second.second;
|
||||
const auto col = static_cast<const ColumnFixedString *>(instr.second.first);
|
||||
const auto length = col->getN();
|
||||
|
||||
memcpy(&out_data[out_offset], &col->getChars()[in_offset], length);
|
||||
out_offset += length;
|
||||
in_offset += length;
|
||||
}
|
||||
else if (instr_type::copy_const_string == instr.first)
|
||||
{
|
||||
const auto col = static_cast<const ColumnConst<String> *>(instr.second.first);
|
||||
const auto & data = col->getData();
|
||||
const auto length = data.size();
|
||||
|
||||
memcpy(&out_data[out_offset], data.data(), length);
|
||||
out_offset += length;
|
||||
}
|
||||
else
|
||||
throw std::logic_error{"unknown instr_type"};
|
||||
}
|
||||
|
||||
out_data[out_offset] = '\0';
|
||||
out_offsets[row] = ++out_offset;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
185
dbms/tests/queries/0_stateless/00078_string_concat.sql
Normal file
185
dbms/tests/queries/0_stateless/00078_string_concat.sql
Normal file
@ -0,0 +1,185 @@
|
||||
select '{ key: fn, value: concat }' == concat('{ key: ', toFixedString('fn', 2), ', value: ', 'concat', ' }');
|
||||
|
||||
select concat('a', 'b') == 'ab';
|
||||
select concat('a', materialize('b')) == 'ab';
|
||||
select concat(materialize('a'), 'b') == 'ab';
|
||||
select concat(materialize('a'), materialize('b')) == 'ab';
|
||||
|
||||
select concat('a', toFixedString('b', 1)) == 'ab';
|
||||
select concat('a', materialize(toFixedString('b', 1))) == 'ab';
|
||||
select concat(materialize('a'), toFixedString('b', 1)) == 'ab';
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1))) == 'ab';
|
||||
|
||||
select concat(toFixedString('a', 1), 'b') == 'ab';
|
||||
select concat(toFixedString('a', 1), materialize('b')) == 'ab';
|
||||
select concat(materialize(toFixedString('a', 1)), 'b') == 'ab';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b')) == 'ab';
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1)) == 'ab';
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1))) == 'ab';
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1)) == 'ab';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1))) == 'ab';
|
||||
|
||||
select concat('a', 'b') == 'ab' from system.numbers limit 5;
|
||||
select concat('a', materialize('b')) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize('a'), 'b') == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize('b')) == 'ab' from system.numbers limit 5;
|
||||
|
||||
select concat('a', toFixedString('b', 1)) == 'ab' from system.numbers limit 5;
|
||||
select concat('a', materialize(toFixedString('b', 1))) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize('a'), toFixedString('b', 1)) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1))) == 'ab' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), 'b') == 'ab' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize('b')) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), 'b') == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b')) == 'ab' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1)) == 'ab' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1))) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1)) == 'ab' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1))) == 'ab' from system.numbers limit 5;
|
||||
|
||||
select concat('a', 'b', 'c') == 'abc';
|
||||
select concat('a', 'b', materialize('c')) == 'abc';
|
||||
select concat('a', materialize('b'), 'c') == 'abc';
|
||||
select concat('a', materialize('b'), materialize('c')) == 'abc';
|
||||
select concat(materialize('a'), 'b', 'c') == 'abc';
|
||||
select concat(materialize('a'), 'b', materialize('c')) == 'abc';
|
||||
select concat(materialize('a'), materialize('b'), 'c') == 'abc';
|
||||
select concat(materialize('a'), materialize('b'), materialize('c')) == 'abc';
|
||||
|
||||
select concat('a', 'b', toFixedString('c', 1)) == 'abc';
|
||||
select concat('a', 'b', materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat('a', materialize('b'), toFixedString('c', 1)) == 'abc';
|
||||
select concat('a', materialize('b'), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize('a'), 'b', toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize('a'), 'b', materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize('a'), materialize('b'), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize('a'), materialize('b'), materialize(toFixedString('c', 1))) == 'abc';
|
||||
|
||||
select concat('a', toFixedString('b', 1), 'c') == 'abc';
|
||||
select concat('a', toFixedString('b', 1), materialize('c')) == 'abc';
|
||||
select concat('a', materialize(toFixedString('b', 1)), 'c') == 'abc';
|
||||
select concat('a', materialize(toFixedString('b', 1)), materialize('c')) == 'abc';
|
||||
select concat(materialize('a'), toFixedString('b', 1), 'c') == 'abc';
|
||||
select concat(materialize('a'), toFixedString('b', 1), materialize('c')) == 'abc';
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), 'c') == 'abc';
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), materialize('c')) == 'abc';
|
||||
|
||||
select concat('a', toFixedString('b', 1), toFixedString('c', 1)) == 'abc';
|
||||
select concat('a', toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat('a', materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc';
|
||||
select concat('a', materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize('a'), toFixedString('b', 1), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize('a'), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc';
|
||||
|
||||
select concat(toFixedString('a', 1), 'b', 'c') == 'abc';
|
||||
select concat(toFixedString('a', 1), 'b', materialize('c')) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize('b'), 'c') == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize('b'), materialize('c')) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', 'c') == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', materialize('c')) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), 'c') == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), materialize('c')) == 'abc';
|
||||
|
||||
select concat(toFixedString('a', 1), 'b', toFixedString('c', 1)) == 'abc';
|
||||
select concat(toFixedString('a', 1), 'b', materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize('b'), toFixedString('c', 1)) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize('b'), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), materialize(toFixedString('c', 1))) == 'abc';
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), 'c') == 'abc';
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), materialize('c')) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), 'c') == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), materialize('c')) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), 'c') == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), materialize('c')) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), 'c') == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), materialize('c')) == 'abc';
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), toFixedString('c', 1)) == 'abc';
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc';
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc';
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc';
|
||||
|
||||
select concat('a', 'b', 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat('a', 'b', materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize('b'), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize('b'), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), 'b', 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), 'b', materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize('b'), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize('b'), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat('a', 'b', toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', 'b', materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize('b'), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize('b'), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), 'b', toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), 'b', materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize('b'), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize('b'), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat('a', toFixedString('b', 1), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat('a', toFixedString('b', 1), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize(toFixedString('b', 1)), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize(toFixedString('b', 1)), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), toFixedString('b', 1), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), toFixedString('b', 1), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat('a', toFixedString('b', 1), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat('a', materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), toFixedString('b', 1), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize('a'), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), 'b', 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), 'b', materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize('b'), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize('b'), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), 'b', toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), 'b', materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize('b'), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize('b'), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), 'b', materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize('b'), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), 'c') == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), materialize('c')) == 'abc' from system.numbers limit 5;
|
||||
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(toFixedString('a', 1), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), toFixedString('b', 1), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), toFixedString('c', 1)) == 'abc' from system.numbers limit 5;
|
||||
select concat(materialize(toFixedString('a', 1)), materialize(toFixedString('b', 1)), materialize(toFixedString('c', 1))) == 'abc' from system.numbers limit 5;
|
Loading…
Reference in New Issue
Block a user