diff --git a/cmake/find_capnp.cmake b/cmake/find_capnp.cmake index 6c064112686..ec591afdc38 100644 --- a/cmake/find_capnp.cmake +++ b/cmake/find_capnp.cmake @@ -29,7 +29,7 @@ if (ENABLE_CAPNP) find_library (CAPNP capnp PATHS ${CAPNP_PATHS}) find_library (CAPNPC capnpc PATHS ${CAPNP_PATHS}) find_library (KJ kj PATHS ${CAPNP_PATHS}) - set (CAPNP_LIBRARY ${CAPNP} ${CAPNPC} ${KJ}) + set (CAPNP_LIBRARY ${CAPNPC} ${CAPNP} ${KJ}) find_path (CAPNP_INCLUDE_DIR NAMES capnp/schema-parser.h PATHS ${CAPNP_INCLUDE_PATHS}) endif () diff --git a/contrib/poco b/contrib/poco index 3df947389e6..d7a4383c4d8 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 3df947389e6d9654919002797bdd86ed190b3963 +Subproject commit d7a4383c4d85b51938b62ed5812bc0935245edb3 diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index c214204d0c2..6cdffb3b5cc 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54407 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 12 CACHE STRING "") -set(VERSION_PATCH 8 CACHE STRING "") -set(VERSION_GITHASH 199d8734f98fa7d04ebf2119431c5f56a7ed4e5a CACHE STRING "") -set(VERSION_DESCRIBE v18.12.8-testing CACHE STRING "") -set(VERSION_STRING 18.12.8 CACHE STRING "") +set(VERSION_PATCH 11 CACHE STRING "") +set(VERSION_GITHASH 1d28a9c510120b07f0719b2f33ccbc21be1e339d CACHE STRING "") +set(VERSION_DESCRIBE v18.12.11-testing CACHE STRING "") +set(VERSION_STRING 18.12.11 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index bd86e2d9ddf..6b011982534 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -37,7 +37,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index 3ba6d76179e..d5c6b6bacf6 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -862,9 +862,9 @@ class ModelFactory public: ModelPtr get(const IDataType & data_type, UInt64 seed, MarkovModelParameters markov_model_params) const { - if (data_type.isInteger()) + if (isInteger(data_type)) { - if (data_type.isUnsignedInteger()) + if (isUnsignedInteger(data_type)) return std::make_unique(seed); else return std::make_unique(seed); diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 9b96cfa00fb..1a2f321fa00 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -213,9 +213,7 @@ void HTTPHandler::processQuery( Context context = server.context(); context.setGlobalContext(server.context()); - /// It will forcibly detach query even if unexpected error ocurred and detachQuery() was not called - /// Normal detaching is happen in BlockIO callbacks - CurrentThread::QueryScope query_scope_holder(context); + CurrentThread::QueryScope query_scope(context); LOG_TRACE(log, "Request URI: " << request.getURI()); diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 4499087ff20..8432db7176d 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -130,6 +130,9 @@ void TCPHandler::runImpl() Stopwatch watch; state.reset(); + /// Initialized later. + std::optional query_scope; + /** An exception during the execution of request (it must be sent over the network to the client). * The client will be able to accept it, if it did not happen while sending another packet and the client has not disconnected yet. */ @@ -152,7 +155,7 @@ void TCPHandler::runImpl() if (!receivePacket()) continue; - CurrentThread::initializeQuery(); + query_scope.emplace(query_context); send_exception_with_stack_trace = query_context.getSettingsRef().calculate_text_stack_trace; @@ -197,6 +200,8 @@ void TCPHandler::runImpl() sendLogs(); sendEndOfStream(); + + query_scope.reset(); state.reset(); } catch (const Exception & e) @@ -265,9 +270,7 @@ void TCPHandler::runImpl() try { - /// It will forcibly detach query even if unexpected error ocсurred and detachQuery() was not called - CurrentThread::detachQueryIfNotDetached(); - + query_scope.reset(); state.reset(); } catch (...) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index edba90adbba..b998ee97216 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -79,7 +79,7 @@ public: if (arguments.size() != 2) throw Exception("Aggregate function " + getName() + " requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!arguments[1]->isUnsignedInteger()) + if (!isUnsignedInteger(arguments[1])) throw Exception("Second argument of aggregate function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); type = arguments.front(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 62794e6f536..c8432fcf7f2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -61,10 +61,10 @@ public: AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments) : kind(kind_) { - if (!arguments[0]->isNumber()) + if (!isNumber(arguments[0])) throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!arguments[1]->isNumber()) + if (!isNumber(arguments[1])) throw Exception{getName() + ": second argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (!arguments[0]->equals(*arguments[1])) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index d863b7116d5..b50bb81f884 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -33,8 +33,6 @@ void ColumnAggregateFunction::addArena(ArenaPtr arena_) MutableColumnPtr ColumnAggregateFunction::convertToValues() const { - const IAggregateFunction * function = func.get(); - /** If the aggregate function returns an unfinalized/unfinished state, * then you just need to copy pointers to it and also shared ownership of data. * @@ -65,33 +63,73 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues() const * AggregateFunction(quantileTiming(0.5), UInt64) * into UInt16 - already finished result of `quantileTiming`. */ - if (const AggregateFunctionState * function_state = typeid_cast(function)) + if (const AggregateFunctionState * function_state = typeid_cast(func.get())) { auto res = createView(); res->set(function_state->getNestedFunction()); - res->getData().assign(getData().begin(), getData().end()); + res->data.assign(data.begin(), data.end()); return res; } - MutableColumnPtr res = function->getReturnType()->createColumn(); - res->reserve(getData().size()); + MutableColumnPtr res = func->getReturnType()->createColumn(); + res->reserve(data.size()); - for (auto val : getData()) - function->insertResultInto(val, *res); + for (auto val : data) + func->insertResultInto(val, *res); return res; } +void ColumnAggregateFunction::ensureOwnership() +{ + if (src) + { + /// We must copy all data from src and take ownership. + size_t size = data.size(); + + Arena & arena = createOrGetArena(); + size_t size_of_state = func->sizeOfData(); + size_t align_of_state = func->alignOfData(); + + size_t rollback_pos = 0; + try + { + for (size_t i = 0; i < size; ++i) + { + ConstAggregateDataPtr old_place = data[i]; + data[i] = arena.alignedAlloc(size_of_state, align_of_state); + func->create(data[i]); + ++rollback_pos; + func->merge(data[i], old_place, &arena); + } + } + catch (...) + { + /// If we failed to take ownership, destroy all temporary data. + + if (!func->hasTrivialDestructor()) + for (size_t i = 0; i < rollback_pos; ++i) + func->destroy(data[i]); + + throw; + } + + /// Now we own all data. + src.reset(); + } +} + + void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start, size_t length) { const ColumnAggregateFunction & from_concrete = static_cast(from); - if (start + length > from_concrete.getData().size()) + if (start + length > from_concrete.data.size()) throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length) + " are out of bound in ColumnAggregateFunction::insertRangeFrom method" " (data.size() = " - + toString(from_concrete.getData().size()) + + toString(from_concrete.data.size()) + ").", ErrorCodes::PARAMETER_OUT_OF_BOUND); @@ -112,14 +150,14 @@ void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start size_t old_size = data.size(); data.resize(old_size + length); - memcpy(&data[old_size], &from_concrete.getData()[start], length * sizeof(data[0])); + memcpy(&data[old_size], &from_concrete.data[start], length * sizeof(data[0])); } } ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_size_hint) const { - size_t size = getData().size(); + size_t size = data.size(); if (size != filter.size()) throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); @@ -127,14 +165,14 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_ return cloneEmpty(); auto res = createView(); - auto & res_data = res->getData(); + auto & res_data = res->data; if (result_size_hint) res_data.reserve(result_size_hint > 0 ? result_size_hint : size); for (size_t i = 0; i < size; ++i) if (filter[i]) - res_data.push_back(getData()[i]); + res_data.push_back(data[i]); /// To save RAM in case of too strong filtering. if (res_data.size() * 2 < res_data.capacity()) @@ -146,7 +184,7 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const { - size_t size = getData().size(); + size_t size = data.size(); if (limit == 0) limit = size; @@ -158,9 +196,9 @@ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limi auto res = createView(); - res->getData().resize(limit); + res->data.resize(limit); for (size_t i = 0; i < limit; ++i) - res->getData()[i] = getData()[perm[i]]; + res->data[i] = data[perm[i]]; return res; } @@ -175,9 +213,9 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray & indexe { auto res = createView(); - res->getData().resize(limit); + res->data.resize(limit); for (size_t i = 0; i < limit; ++i) - res->getData()[i] = getData()[indexes[i]]; + res->data[i] = data[indexes[i]]; return res; } @@ -188,14 +226,14 @@ INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction) void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const { WriteBufferFromOwnString wbuf; - func->serialize(getData()[n], wbuf); + func->serialize(data[n], wbuf); hash.update(wbuf.str().c_str(), wbuf.str().size()); } /// NOTE: Highly overestimates size of a column if it was produced in AggregatingBlockInputStream (it contains size of other columns) size_t ColumnAggregateFunction::byteSize() const { - size_t res = getData().size() * sizeof(getData()[0]); + size_t res = data.size() * sizeof(data[0]); for (const auto & arena : arenas) res += arena->size(); @@ -207,7 +245,7 @@ size_t ColumnAggregateFunction::byteSize() const /// Like byteSize(), highly overestimates size size_t ColumnAggregateFunction::allocatedBytes() const { - size_t res = getData().allocated_bytes(); + size_t res = data.allocated_bytes(); for (const auto & arena : arenas) res += arena->size(); @@ -225,7 +263,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const Field field = String(); { WriteBufferFromString buffer(field.get()); - func->serialize(getData()[n], buffer); + func->serialize(data[n], buffer); } return field; } @@ -235,18 +273,19 @@ void ColumnAggregateFunction::get(size_t n, Field & res) const res = String(); { WriteBufferFromString buffer(res.get()); - func->serialize(getData()[n], buffer); + func->serialize(data[n], buffer); } } StringRef ColumnAggregateFunction::getDataAt(size_t n) const { - return StringRef(reinterpret_cast(&getData()[n]), sizeof(getData()[n])); + return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); } void ColumnAggregateFunction::insertData(const char * pos, size_t /*length*/) { - getData().push_back(*reinterpret_cast(pos)); + ensureOwnership(); + data.push_back(*reinterpret_cast(pos)); } void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n) @@ -254,24 +293,26 @@ void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n) /// Must create new state of aggregate function and take ownership of it, /// because ownership of states of aggregate function cannot be shared for individual rows, /// (only as a whole, see comment above). + ensureOwnership(); insertDefault(); insertMergeFrom(from, n); } void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place) { + ensureOwnership(); insertDefault(); insertMergeFrom(place); } void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place) { - func->merge(getData().back(), place, &createOrGetArena()); + func->merge(data.back(), place, &createOrGetArena()); } void ColumnAggregateFunction::insertMergeFrom(const IColumn & from, size_t n) { - insertMergeFrom(static_cast(from).getData()[n]); + insertMergeFrom(static_cast(from).data[n]); } Arena & ColumnAggregateFunction::createOrGetArena() @@ -281,47 +322,54 @@ Arena & ColumnAggregateFunction::createOrGetArena() return *arenas.back().get(); } + +static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Arena & arena, IAggregateFunction * func) +{ + data.push_back(arena.alignedAlloc(func->sizeOfData(), func->alignOfData())); + try + { + func->create(data.back()); + } + catch (...) + { + data.pop_back(); + throw; + } +} + + void ColumnAggregateFunction::insert(const Field & x) { - IAggregateFunction * function = func.get(); - + ensureOwnership(); Arena & arena = createOrGetArena(); - - getData().push_back(arena.alignedAlloc(function->sizeOfData(), function->alignOfData())); - function->create(getData().back()); + pushBackAndCreateState(data, arena, func.get()); ReadBufferFromString read_buffer(x.get()); - function->deserialize(getData().back(), read_buffer, &arena); + func->deserialize(data.back(), read_buffer, &arena); } void ColumnAggregateFunction::insertDefault() { - IAggregateFunction * function = func.get(); - + ensureOwnership(); Arena & arena = createOrGetArena(); - - getData().push_back(arena.alignedAlloc(function->sizeOfData(), function->alignOfData())); - function->create(getData().back()); + pushBackAndCreateState(data, arena, func.get()); } StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & dst, const char *& begin) const { - IAggregateFunction * function = func.get(); WriteBufferFromArena out(dst, begin); - function->serialize(getData()[n], out); + func->serialize(data[n], out); return out.finish(); } const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * src_arena) { - IAggregateFunction * function = func.get(); + ensureOwnership(); /** Parameter "src_arena" points to Arena, from which we will deserialize the state. * And "dst_arena" is another Arena, that aggregate function state will use to store its data. */ Arena & dst_arena = createOrGetArena(); - - getData().push_back(dst_arena.alignedAlloc(function->sizeOfData(), function->alignOfData())); - function->create(getData().back()); + pushBackAndCreateState(data, dst_arena, func.get()); /** We will read from src_arena. * There is no limit for reading - it is assumed, that we can read all that we need after src_arena pointer. @@ -331,7 +379,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * * Probably this will not work under UBSan. */ ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits::max() - src_arena); - function->deserialize(getData().back(), read_buffer, &dst_arena); + func->deserialize(data.back(), read_buffer, &dst_arena); return read_buffer.position(); } @@ -358,7 +406,7 @@ ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) c return cloneEmpty(); auto res = createView(); - auto & res_data = res->getData(); + auto & res_data = res->data; res_data.reserve(offsets.back()); IColumn::Offset prev_offset = 0; @@ -399,7 +447,7 @@ MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns void ColumnAggregateFunction::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const { - size_t s = getData().size(); + size_t s = data.size(); res.resize(s); for (size_t i = 0; i < s; ++i) res[i] = i; diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index 674d98f8892..cd352007095 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -74,6 +74,11 @@ private: return res; } + /// If we have another column as a source (owner of data), copy all data to ourself and reset source. + /// This is needed before inserting new elements, because we must own these elements (to destroy them in destructor), + /// but ownership of different elements cannot be mixed by different columns. + void ensureOwnership(); + ColumnAggregateFunction(const AggregateFunctionPtr & func_) : func(func_) { diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 642d3da2327..c2c17c17ed7 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -137,7 +137,7 @@ private: /** Non-constant arrays of constant values are quite rare. * Most functions can not work with them, and does not create such columns as a result. - * An exception is the function `replicate`(see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions. + * An exception is the function `replicate` (see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions. * Only for its sake is the implementation of the `replicate` method for ColumnArray(ColumnConst). */ ColumnPtr replicateConst(const Offsets & replicate_offsets) const; diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index ced8713c63d..4e724c995c4 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -395,6 +395,7 @@ namespace ErrorCodes extern const int UNKNOWN_JOIN_STRICTNESS = 418; extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN = 419; extern const int CANNOT_UPDATE_COLUMN = 420; + extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES = 421; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index c3e77c11a92..716de954974 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -317,7 +317,7 @@ Columns Block::getColumns() const } -MutableColumns Block::mutateColumns() const +MutableColumns Block::mutateColumns() { size_t num_columns = data.size(); MutableColumns columns(num_columns); diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 23c4b513f5f..27b107eeb09 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -115,7 +115,7 @@ public: MutableColumns cloneEmptyColumns() const; /** Get columns from block for mutation. Columns in block will be nullptr. */ - MutableColumns mutateColumns() const; + MutableColumns mutateColumns(); /** Replace columns in a block */ void setColumns(MutableColumns && columns); diff --git a/dbms/src/Core/DecimalComparison.h b/dbms/src/Core/DecimalComparison.h index 64d3f9b8302..713f6a6e770 100644 --- a/dbms/src/Core/DecimalComparison.h +++ b/dbms/src/Core/DecimalComparison.h @@ -20,7 +20,7 @@ namespace ErrorCodes } /// -inline bool allowDecimalComparison(const IDataType * left_type, const IDataType * right_type) +inline bool allowDecimalComparison(const DataTypePtr & left_type, const DataTypePtr & right_type) { if (isDecimal(left_type)) { diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index 581ced9df36..5e2cd47f440 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -59,7 +59,7 @@ template <> struct TypeName { static const char * get() { return "Strin enum class TypeIndex { - None = 0, + Nothing = 0, UInt8, UInt16, UInt32, @@ -84,6 +84,12 @@ enum class TypeIndex UUID, Array, Tuple, + Set, + Interval, + Nullable, + Function, + AggregateFunction, + LowCardinality, }; template struct TypeId; diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index 389dbf5bec5..e43aa51e617 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -7,22 +8,6 @@ namespace DB { -static void finalize(Block & block) -{ - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnWithTypeAndName & current = block.getByPosition(i); - const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); - - if (unfinalized_type) - { - current.type = unfinalized_type->getReturnType(); - if (current.column) - current.column = typeid_cast(*current.column).convertToValues(); - } - } -} - RollupBlockInputStream::RollupBlockInputStream( const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : aggregator(params_), keys(params_.keys) @@ -36,7 +21,7 @@ RollupBlockInputStream::RollupBlockInputStream( Block RollupBlockInputStream::getHeader() const { Block res = children.at(0)->getHeader(); - finalize(res); + finalizeBlock(res); return res; } @@ -58,7 +43,7 @@ Block RollupBlockInputStream::readImpl() rollup_block = aggregator.mergeBlocks(rollup_blocks, false); Block finalized = rollup_block; - finalize(finalized); + finalizeBlock(finalized); return finalized; } @@ -66,7 +51,7 @@ Block RollupBlockInputStream::readImpl() current_key = keys.size() - 1; rollup_block = block; - finalize(block); + finalizeBlock(block); return block; } diff --git a/dbms/src/DataStreams/SquashingBlockInputStream.cpp b/dbms/src/DataStreams/SquashingBlockInputStream.cpp index 61af8e1a0f0..f1d14b313e7 100644 --- a/dbms/src/DataStreams/SquashingBlockInputStream.cpp +++ b/dbms/src/DataStreams/SquashingBlockInputStream.cpp @@ -4,9 +4,9 @@ namespace DB { -SquashingBlockInputStream::SquashingBlockInputStream(const BlockInputStreamPtr & src, - size_t min_block_size_rows, size_t min_block_size_bytes) - : transform(min_block_size_rows, min_block_size_bytes) +SquashingBlockInputStream::SquashingBlockInputStream( + const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes) + : header(src->getHeader()), transform(min_block_size_rows, min_block_size_bytes) { children.emplace_back(src); } @@ -23,9 +23,13 @@ Block SquashingBlockInputStream::readImpl() if (!block) all_read = true; - SquashingTransform::Result result = transform.add(std::move(block)); + SquashingTransform::Result result = transform.add(block.mutateColumns()); if (result.ready) - return result.block; + { + if (result.columns.empty()) + return {}; + return header.cloneWithColumns(std::move(result.columns)); + } } } diff --git a/dbms/src/DataStreams/SquashingBlockInputStream.h b/dbms/src/DataStreams/SquashingBlockInputStream.h index 0f9a01d65e8..feac5e2cdcb 100644 --- a/dbms/src/DataStreams/SquashingBlockInputStream.h +++ b/dbms/src/DataStreams/SquashingBlockInputStream.h @@ -16,12 +16,13 @@ public: String getName() const override { return "Squashing"; } - Block getHeader() const override { return children.at(0)->getHeader(); } + Block getHeader() const override { return header; } protected: Block readImpl() override; private: + Block header; SquashingTransform transform; bool all_read = false; }; diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.cpp b/dbms/src/DataStreams/SquashingBlockOutputStream.cpp index ff340926c90..b811dc3e5d1 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.cpp +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.cpp @@ -5,16 +5,16 @@ namespace DB { SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr & dst, size_t min_block_size_rows, size_t min_block_size_bytes) - : output(dst), transform(min_block_size_rows, min_block_size_bytes) + : output(dst), header(output->getHeader()), transform(min_block_size_rows, min_block_size_bytes) { } void SquashingBlockOutputStream::write(const Block & block) { - SquashingTransform::Result result = transform.add(Block(block)); + SquashingTransform::Result result = transform.add(Block(block).mutateColumns()); if (result.ready) - output->write(result.block); + output->write(header.cloneWithColumns(std::move(result.columns))); } @@ -26,8 +26,8 @@ void SquashingBlockOutputStream::finalize() all_written = true; SquashingTransform::Result result = transform.add({}); - if (result.ready && result.block) - output->write(result.block); + if (result.ready && !result.columns.empty()) + output->write(header.cloneWithColumns(std::move(result.columns))); } diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.h b/dbms/src/DataStreams/SquashingBlockOutputStream.h index 9e660de59f1..bd19c5e2cdc 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.h +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.h @@ -14,7 +14,7 @@ class SquashingBlockOutputStream : public IBlockOutputStream public: SquashingBlockOutputStream(BlockOutputStreamPtr & dst, size_t min_block_size_rows, size_t min_block_size_bytes); - Block getHeader() const override { return output->getHeader(); } + Block getHeader() const override { return header; } void write(const Block & block) override; void flush() override; @@ -26,6 +26,7 @@ public: private: BlockOutputStreamPtr output; + Block header; SquashingTransform transform; bool all_written = false; diff --git a/dbms/src/DataStreams/SquashingTransform.cpp b/dbms/src/DataStreams/SquashingTransform.cpp index d31ff6f8063..abac72e79bd 100644 --- a/dbms/src/DataStreams/SquashingTransform.cpp +++ b/dbms/src/DataStreams/SquashingTransform.cpp @@ -10,37 +10,38 @@ SquashingTransform::SquashingTransform(size_t min_block_size_rows, size_t min_bl } -SquashingTransform::Result SquashingTransform::add(Block && block) +SquashingTransform::Result SquashingTransform::add(MutableColumns && columns) { - if (!block) - return Result(std::move(accumulated_block)); + /// End of input stream. + if (columns.empty()) + return Result(std::move(accumulated_columns)); /// Just read block is alredy enough. - if (isEnoughSize(block.rows(), block.bytes())) + if (isEnoughSize(columns)) { /// If no accumulated data, return just read block. - if (!accumulated_block) - return Result(std::move(block)); + if (accumulated_columns.empty()) + return Result(std::move(columns)); - /// Return accumulated data (may be it has small size) and place new block to accumulated data. - accumulated_block.swap(block); - return Result(std::move(block)); + /// Return accumulated data (maybe it has small size) and place new block to accumulated data. + columns.swap(accumulated_columns); + return Result(std::move(columns)); } /// Accumulated block is already enough. - if (accumulated_block && isEnoughSize(accumulated_block.rows(), accumulated_block.bytes())) + if (!accumulated_columns.empty() && isEnoughSize(accumulated_columns)) { /// Return accumulated data and place new block to accumulated data. - accumulated_block.swap(block); - return Result(std::move(block)); + columns.swap(accumulated_columns); + return Result(std::move(columns)); } - append(std::move(block)); + append(std::move(columns)); - if (isEnoughSize(accumulated_block.rows(), accumulated_block.bytes())) + if (isEnoughSize(accumulated_columns)) { - Block res; - res.swap(accumulated_block); + MutableColumns res; + res.swap(accumulated_columns); return Result(std::move(res)); } @@ -49,23 +50,35 @@ SquashingTransform::Result SquashingTransform::add(Block && block) } -void SquashingTransform::append(Block && block) +void SquashingTransform::append(MutableColumns && columns) { - if (!accumulated_block) + if (accumulated_columns.empty()) { - accumulated_block = std::move(block); + accumulated_columns = std::move(columns); return; } - size_t columns = block.columns(); - size_t rows = block.rows(); + for (size_t i = 0, size = columns.size(); i < size; ++i) + accumulated_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); +} - for (size_t i = 0; i < columns; ++i) + +bool SquashingTransform::isEnoughSize(const MutableColumns & columns) +{ + size_t rows = 0; + size_t bytes = 0; + + for (const auto & column : columns) { - MutableColumnPtr mutable_column = (*std::move(accumulated_block.getByPosition(i).column)).mutate(); - mutable_column->insertRangeFrom(*block.getByPosition(i).column, 0, rows); - accumulated_block.getByPosition(i).column = std::move(mutable_column); + if (!rows) + rows = column->size(); + else if (rows != column->size()) + throw Exception("Sizes of columns doesn't match", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + bytes += column->byteSize(); } + + return isEnoughSize(rows, bytes); } diff --git a/dbms/src/DataStreams/SquashingTransform.h b/dbms/src/DataStreams/SquashingTransform.h index 25e75a6438d..519c0e1ae4b 100644 --- a/dbms/src/DataStreams/SquashingTransform.h +++ b/dbms/src/DataStreams/SquashingTransform.h @@ -29,25 +29,26 @@ public: struct Result { bool ready = false; - Block block; + MutableColumns columns; Result(bool ready_) : ready(ready_) {} - Result(Block && block_) : ready(true), block(std::move(block_)) {} + Result(MutableColumns && columns) : ready(true), columns(std::move(columns)) {} }; /** Add next block and possibly returns squashed block. * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. */ - Result add(Block && block); + Result add(MutableColumns && columns); private: size_t min_block_size_rows; size_t min_block_size_bytes; - Block accumulated_block; + MutableColumns accumulated_columns; - void append(Block && block); + void append(MutableColumns && columns); + bool isEnoughSize(const MutableColumns & columns); bool isEnoughSize(size_t rows, size_t bytes) const; }; diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index f87b2d63b11..00f3f55c3a8 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -76,7 +76,7 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( } else { - bool is_agg_func = checkDataType(column.type.get()); + bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); if (!column.type->isSummable() && !is_agg_func) { column_numbers_not_to_aggregate.push_back(i); @@ -273,7 +273,7 @@ Block SummingSortedBlockInputStream::readImpl() for (auto & desc : columns_to_aggregate) { // Wrap aggregated columns in a tuple to match function signature - if (!desc.is_agg_func_type && checkDataType(desc.function->getReturnType().get())) + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) { size_t tuple_size = desc.column_numbers.size(); MutableColumns tuple_columns(tuple_size); @@ -292,7 +292,7 @@ Block SummingSortedBlockInputStream::readImpl() /// Place aggregation results into block. for (auto & desc : columns_to_aggregate) { - if (!desc.is_agg_func_type && checkDataType(desc.function->getReturnType().get())) + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) { /// Unpack tuple into block. size_t tuple_size = desc.column_numbers.size(); diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp index e964065822c..103d880f1d3 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -53,23 +54,6 @@ TotalsHavingBlockInputStream::TotalsHavingBlockInputStream( } -static void finalize(Block & block) -{ - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnWithTypeAndName & current = block.getByPosition(i); - const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); - - if (unfinalized_type) - { - current.type = unfinalized_type->getReturnType(); - if (current.column) - current.column = typeid_cast(*current.column).convertToValues(); - } - } -} - - Block TotalsHavingBlockInputStream::getTotals() { if (!totals) @@ -87,7 +71,7 @@ Block TotalsHavingBlockInputStream::getTotals() } totals = children.at(0)->getHeader().cloneWithColumns(std::move(current_totals)); - finalize(totals); + finalizeBlock(totals); } if (totals && expression) @@ -101,7 +85,7 @@ Block TotalsHavingBlockInputStream::getHeader() const { Block res = children.at(0)->getHeader(); if (final) - finalize(res); + finalizeBlock(res); if (expression) expression->execute(res); return res; @@ -129,7 +113,7 @@ Block TotalsHavingBlockInputStream::readImpl() finalized = block; if (final) - finalize(finalized); + finalizeBlock(finalized); total_keys += finalized.rows(); diff --git a/dbms/src/DataStreams/finalizeBlock.cpp b/dbms/src/DataStreams/finalizeBlock.cpp new file mode 100644 index 00000000000..50fbaf2bfe1 --- /dev/null +++ b/dbms/src/DataStreams/finalizeBlock.cpp @@ -0,0 +1,24 @@ +#include +#include +#include +#include + + +namespace DB +{ + void finalizeBlock(Block & block) + { + for (size_t i = 0; i < block.columns(); ++i) + { + ColumnWithTypeAndName & current = block.getByPosition(i); + const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); + + if (unfinalized_type) + { + current.type = unfinalized_type->getReturnType(); + if (current.column) + current.column = typeid_cast(*current.column).convertToValues(); + } + } + } +} diff --git a/dbms/src/DataStreams/finalizeBlock.h b/dbms/src/DataStreams/finalizeBlock.h new file mode 100644 index 00000000000..3c81ddae1c7 --- /dev/null +++ b/dbms/src/DataStreams/finalizeBlock.h @@ -0,0 +1,9 @@ +#pragma once + +#include + +namespace DB +{ + /// Converts aggregate function columns with non-finalized states to final values + void finalizeBlock(Block & block); +} diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.h b/dbms/src/DataTypes/DataTypeAggregateFunction.h index cd3fdb468cc..d68f460dea6 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.h @@ -30,8 +30,8 @@ public: AggregateFunctionPtr getFunction() const { return function; } std::string getName() const override; - const char * getFamilyName() const override { return "AggregateFunction"; } + TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } bool canBeInsideNullable() const override { return false; } diff --git a/dbms/src/DataTypes/DataTypeDate.h b/dbms/src/DataTypes/DataTypeDate.h index 4e71065e8e3..89d773149c5 100644 --- a/dbms/src/DataTypes/DataTypeDate.h +++ b/dbms/src/DataTypes/DataTypeDate.h @@ -23,7 +23,6 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; bool canBeUsedAsVersion() const override { return true; } - bool isDateOrDateTime() const override { return true; } bool canBeInsideNullable() const override { return true; } bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypeDateTime.h b/dbms/src/DataTypes/DataTypeDateTime.h index aa6f9950563..95b120c6c9d 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.h +++ b/dbms/src/DataTypes/DataTypeDateTime.h @@ -48,7 +48,6 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; bool canBeUsedAsVersion() const override { return true; } - bool isDateOrDateTime() const override { return true; } bool canBeInsideNullable() const override { return true; } bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index c514e33d2f6..43e4c578f76 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -30,7 +30,6 @@ public: bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } bool haveMaximumSizeOfValue() const override { return true; } bool isCategorial() const override { return true; } - bool isEnum() const override { return true; } bool canBeInsideNullable() const override { return true; } bool isComparable() const override { return true; } }; diff --git a/dbms/src/DataTypes/DataTypeFixedString.h b/dbms/src/DataTypes/DataTypeFixedString.h index 83934de1f41..34a3f02c3c1 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.h +++ b/dbms/src/DataTypes/DataTypeFixedString.h @@ -77,7 +77,7 @@ public: bool haveSubtypes() const override { return false; } bool isComparable() const override { return true; } bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool isFixedString() const override { return true; } + bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; } bool haveMaximumSizeOfValue() const override { return true; } size_t getSizeOfValueInMemory() const override { return n; } bool isCategorial() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeFunction.h b/dbms/src/DataTypes/DataTypeFunction.h index a84d14ae114..73089400a18 100644 --- a/dbms/src/DataTypes/DataTypeFunction.h +++ b/dbms/src/DataTypes/DataTypeFunction.h @@ -24,6 +24,7 @@ public: std::string getName() const override; const char * getFamilyName() const override { return "Function"; } + TypeIndex getTypeId() const override { return TypeIndex::Function; } const DataTypes & getArgumentTypes() const { diff --git a/dbms/src/DataTypes/DataTypeInterval.h b/dbms/src/DataTypes/DataTypeInterval.h index 749c6f95844..afbcf2d6a45 100644 --- a/dbms/src/DataTypes/DataTypeInterval.h +++ b/dbms/src/DataTypes/DataTypeInterval.h @@ -55,6 +55,7 @@ public: std::string getName() const override { return std::string("Interval") + kindToString(); } const char * getFamilyName() const override { return "Interval"; } + TypeIndex getTypeId() const override { return TypeIndex::Interval; } bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypeNothing.h b/dbms/src/DataTypes/DataTypeNothing.h index ea4bf211889..e9421fb15e8 100644 --- a/dbms/src/DataTypes/DataTypeNothing.h +++ b/dbms/src/DataTypes/DataTypeNothing.h @@ -16,6 +16,7 @@ public: static constexpr bool is_parametric = false; const char * getFamilyName() const override { return "Nothing"; } + TypeIndex getTypeId() const override { return TypeIndex::Nothing; } MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index a331d42738c..a8bb00862aa 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -16,6 +16,7 @@ public: explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } + TypeIndex getTypeId() const override { return TypeIndex::Nullable; } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; diff --git a/dbms/src/DataTypes/DataTypeSet.h b/dbms/src/DataTypes/DataTypeSet.h index a9df927d7d5..59ed70e4c35 100644 --- a/dbms/src/DataTypes/DataTypeSet.h +++ b/dbms/src/DataTypes/DataTypeSet.h @@ -14,6 +14,7 @@ class DataTypeSet final : public IDataTypeDummy public: static constexpr bool is_parametric = true; const char * getFamilyName() const override { return "Set"; } + TypeIndex getTypeId() const override { return TypeIndex::Set; } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool isParametric() const override { return true; } }; diff --git a/dbms/src/DataTypes/DataTypeString.h b/dbms/src/DataTypes/DataTypeString.h index 2a54808cfd8..b2e36e30cff 100644 --- a/dbms/src/DataTypes/DataTypeString.h +++ b/dbms/src/DataTypes/DataTypeString.h @@ -59,7 +59,6 @@ public: bool isComparable() const override { return true; } bool canBeComparedWithCollation() const override { return true; } bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool isString() const override { return true; } bool isCategorial() const override { return true; } bool canBeInsideNullable() const override { return true; } }; diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 1fcb6caaf68..2053193a7af 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -42,9 +42,9 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_) if (dictionary_type->isNullable()) inner_type = static_cast(*dictionary_type).getNestedType(); - if (!inner_type->isStringOrFixedString() - && !inner_type->isDateOrDateTime() - && !inner_type->isNumber()) + if (!isStringOrFixedString(inner_type) + && !isDateOrDateTime(inner_type) + && !isNumber(inner_type)) throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got " + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -769,15 +769,15 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDat if (auto * nullable_type = typeid_cast(&keys_type)) type = nullable_type->getNestedType().get(); - if (type->isString()) + if (isString(type)) return creator((ColumnString *)(nullptr)); - if (type->isFixedString()) + if (isFixedString(type)) return creator((ColumnFixedString *)(nullptr)); if (typeid_cast(type)) return creator((ColumnVector *)(nullptr)); if (typeid_cast(type)) return creator((ColumnVector *)(nullptr)); - if (type->isNumber()) + if (isNumber(type)) { MutableColumnUniquePtr column; TypeListNumbers::forEach(CreateColumnVector(column, *type, creator)); diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index faf74fefc25..558660faf29 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -20,6 +20,7 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } + TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; @@ -126,20 +127,13 @@ public: bool isSummable() const override { return dictionary_type->isSummable(); } bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); } bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); } - bool isNumber() const override { return false; } - bool isInteger() const override { return false; } - bool isUnsignedInteger() const override { return false; } - bool isDateOrDateTime() const override { return false; } bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); } bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); } bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool isString() const override { return false; } - bool isFixedString() const override { return false; } bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); } size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); } size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); } bool isCategorial() const override { return false; } - bool isEnum() const override { return false; } bool isNullable() const override { return false; } bool onlyNull() const override { return false; } bool withDictionary() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 403e11c8b76..9c6fcf02fbe 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -15,10 +15,7 @@ class DataTypeNumber final : public DataTypeNumberBase bool canBeUsedAsVersion() const override { return true; } bool isSummable() const override { return true; } bool canBeUsedInBitOperations() const override { return true; } - bool isUnsignedInteger() const override { return isInteger() && std::is_unsigned_v; } bool canBeUsedInBooleanContext() const override { return true; } - bool isNumber() const override { return true; } - bool isInteger() const override { return std::is_integral_v; } bool canBeInsideNullable() const override { return true; } }; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index fd8e52cc717..b08b1811151 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -46,7 +46,7 @@ public: virtual const char * getFamilyName() const = 0; /// Unique type number or zero - virtual TypeIndex getTypeId() const { return TypeIndex::None; } + virtual TypeIndex getTypeId() const = 0; /** Binary serialization for range of values in column - for writing to disk/network, etc. * @@ -342,17 +342,6 @@ public: */ virtual bool canBeUsedInBooleanContext() const { return false; } - /** Integers, floats, not Nullable. Not Enums. Not Date/DateTime. - */ - virtual bool isNumber() const { return false; } - - /** Integers. Not Nullable. Not Enums. Not Date/DateTime. - */ - virtual bool isInteger() const { return false; } - virtual bool isUnsignedInteger() const { return false; } - - virtual bool isDateOrDateTime() const { return false; } - /** Numbers, Enums, Date, DateTime. Not nullable. */ virtual bool isValueRepresentedByNumber() const { return false; } @@ -376,13 +365,9 @@ public: virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const { - return isValueRepresentedByNumber() || isFixedString(); + return isValueRepresentedByNumber(); } - virtual bool isString() const { return false; } - virtual bool isFixedString() const { return false; } - virtual bool isStringOrFixedString() const { return isString() || isFixedString(); } - /** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types. * Counterexamples: String, Array. * It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state. @@ -401,8 +386,6 @@ public: */ virtual bool isCategorial() const { return false; } - virtual bool isEnum() const { return false; } - virtual bool isNullable() const { return false; } /** Is this type can represent only NULL value? (It also implies isNullable) @@ -423,11 +406,20 @@ public: }; -struct DataTypeExtractor +/// Some sugar to check data type of IDataType +struct WhichDataType { TypeIndex idx; - DataTypeExtractor(const IDataType * data_type) + WhichDataType(const IDataType & data_type) + : idx(data_type.getTypeId()) + {} + + WhichDataType(const IDataType * data_type) + : idx(data_type->getTypeId()) + {} + + WhichDataType(const DataTypePtr & data_type) : idx(data_type->getTypeId()) {} @@ -437,6 +429,7 @@ struct DataTypeExtractor bool isUInt64() const { return idx == TypeIndex::UInt64; } bool isUInt128() const { return idx == TypeIndex::UInt128; } bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128(); } + bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); } bool isInt8() const { return idx == TypeIndex::Int8; } bool isInt16() const { return idx == TypeIndex::Int16; } @@ -444,6 +437,7 @@ struct DataTypeExtractor bool isInt64() const { return idx == TypeIndex::Int64; } bool isInt128() const { return idx == TypeIndex::Int128; } bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128(); } + bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); } bool isDecimal32() const { return idx == TypeIndex::Decimal32; } bool isDecimal64() const { return idx == TypeIndex::Decimal64; } @@ -469,27 +463,69 @@ struct DataTypeExtractor bool isUUID() const { return idx == TypeIndex::UUID; } bool isArray() const { return idx == TypeIndex::Array; } bool isTuple() const { return idx == TypeIndex::Tuple; } + bool isSet() const { return idx == TypeIndex::Set; } + bool isInterval() const { return idx == TypeIndex::Interval; } + + bool isNothing() const { return idx == TypeIndex::Nothing; } + bool isNullable() const { return idx == TypeIndex::Nullable; } + bool isFunction() const { return idx == TypeIndex::Function; } + bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } }; -/// IDataType helpers (alternative for IDataType virtual methods) +/// IDataType helpers (alternative for IDataType virtual methods with single point of truth) -inline bool isEnum(const IDataType * data_type) +inline bool isDateOrDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateOrDateTime(); } +inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_type).isEnum(); } +inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); } +inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); } +inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); } + +template +inline bool isUnsignedInteger(const T & data_type) { - return DataTypeExtractor(data_type).isEnum(); + return WhichDataType(data_type).isUInt(); } -inline bool isDecimal(const IDataType * data_type) +template +inline bool isInteger(const T & data_type) { - return DataTypeExtractor(data_type).isDecimal(); -} - -inline bool isNotDecimalButComparableToDecimal(const IDataType * data_type) -{ - DataTypeExtractor which(data_type); + WhichDataType which(data_type); return which.isInt() || which.isUInt(); } -inline bool isCompilableType(const IDataType * data_type) +template +inline bool isNumber(const T & data_type) +{ + WhichDataType which(data_type); + return which.isInt() || which.isUInt() || which.isFloat(); +} + +template +inline bool isString(const T & data_type) +{ + return WhichDataType(data_type).isString(); +} + +template +inline bool isFixedString(const T & data_type) +{ + return WhichDataType(data_type).isFixedString(); +} + +template +inline bool isStringOrFixedString(const T & data_type) +{ + return WhichDataType(data_type).isStringOrFixedString(); +} + + +inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type) +{ + WhichDataType which(data_type); + return which.isInt() || which.isUInt(); +} + +inline bool isCompilableType(const DataTypePtr & data_type) { return data_type->isValueRepresentedByNumber() && !isDecimal(data_type); } diff --git a/dbms/src/DataTypes/Native.h b/dbms/src/DataTypes/Native.h index 7dfc6f7a292..cf24a1b7b43 100644 --- a/dbms/src/DataTypes/Native.h +++ b/dbms/src/DataTypes/Native.h @@ -66,6 +66,21 @@ static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDa return nullptr; } +static inline bool canBeNativeType(const IDataType & type) +{ + if (auto * nullable = typeid_cast(&type)) + return canBeNativeType(*nullable->getNestedType()); + + return typeIsEither(type) + || typeIsEither(type) + || typeIsEither(type) + || typeIsEither(type) + || typeIsEither(type) + || typeIsEither(type) + || typeIsEither(type) + || typeid_cast(&type); +} + static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type) { return toNativeType(builder, *type); diff --git a/dbms/src/DataTypes/getMostSubtype.cpp b/dbms/src/DataTypes/getMostSubtype.cpp index 905ad499d66..8fb96f5d366 100644 --- a/dbms/src/DataTypes/getMostSubtype.cpp +++ b/dbms/src/DataTypes/getMostSubtype.cpp @@ -213,7 +213,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth for (const auto & type : types) { - if (type->isFixedString()) + if (isFixedString(type)) { have_string = true; if (!fixed_string_type) @@ -221,7 +221,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth else if (!type->equals(*fixed_string_type)) return getNothingOrThrow(" because some of them are FixedStrings with different length"); } - else if (type->isString()) + else if (isString(type)) have_string = true; else all_strings = false; @@ -243,7 +243,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth for (const auto & type : types) { - if (type->isDateOrDateTime()) + if (isDateOrDateTime(type)) have_date_or_datetime = true; else all_date_or_datetime = false; diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 91fa37564ba..8ca3f10ef2a 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -289,7 +289,7 @@ void ComplexKeyHashedDictionary::updateData() auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); - while (const auto block = stream->read()) + while (Block block = stream->read()) { const auto saved_key_column_ptrs = ext::map(ext::range(0, keys_size), [&](const size_t key_idx) { diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index 8884d83bbef..73eaaef51e0 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -315,7 +315,7 @@ void FlatDictionary::updateData() auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); - while (const auto block = stream->read()) + while (Block block = stream->read()) { const auto &saved_id_column = *saved_block->safeGetByPosition(0).column; const auto &update_id_column = *block.safeGetByPosition(0).column; diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 2008f2a33a7..23f87a41e73 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -307,7 +307,7 @@ void HashedDictionary::updateData() auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); - while (const auto block = stream->read()) + while (Block block = stream->read()) { const auto &saved_id_column = *saved_block->safeGetByPosition(0).column; const auto &update_id_column = *block.safeGetByPosition(0).column; diff --git a/dbms/src/Formats/CSVRowInputStream.cpp b/dbms/src/Formats/CSVRowInputStream.cpp index 5cc1b0e104d..ca8f9514312 100644 --- a/dbms/src/Formats/CSVRowInputStream.cpp +++ b/dbms/src/Formats/CSVRowInputStream.cpp @@ -227,7 +227,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, if (curr_position < prev_position) throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime()) + if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i])) { /// An empty string instead of a value. if (curr_position == prev_position) diff --git a/dbms/src/Formats/TabSeparatedRowInputStream.cpp b/dbms/src/Formats/TabSeparatedRowInputStream.cpp index 9a22cf09237..b843c14bd66 100644 --- a/dbms/src/Formats/TabSeparatedRowInputStream.cpp +++ b/dbms/src/Formats/TabSeparatedRowInputStream.cpp @@ -195,7 +195,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & if (curr_position < prev_position) throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime()) + if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i])) { /// An empty string instead of a value. if (curr_position == prev_position) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 108b5f1800d..2e0c6615676 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -32,46 +32,6 @@ generate_function_register(Arithmetic FunctionIntExp10 ) -generate_function_register(Array - FunctionArray - FunctionArrayElement - FunctionHas - FunctionIndexOf - FunctionCountEqual - FunctionArrayEnumerate - FunctionArrayEnumerateUniq - FunctionArrayEnumerateDense - FunctionArrayUniq - FunctionArrayDistinct - FunctionEmptyArrayUInt8 - FunctionEmptyArrayUInt16 - FunctionEmptyArrayUInt32 - FunctionEmptyArrayUInt64 - FunctionEmptyArrayInt8 - FunctionEmptyArrayInt16 - FunctionEmptyArrayInt32 - FunctionEmptyArrayInt64 - FunctionEmptyArrayFloat32 - FunctionEmptyArrayFloat64 - FunctionEmptyArrayDate - FunctionEmptyArrayDateTime - FunctionEmptyArrayString - FunctionEmptyArrayToSingle - FunctionRange - FunctionArrayReduce - FunctionArrayReverse - FunctionArrayConcat - FunctionArraySlice - FunctionArrayPushBack - FunctionArrayPushFront - FunctionArrayPopBack - FunctionArrayPopFront - FunctionArrayHasAll - FunctionArrayHasAny - FunctionArrayIntersect - FunctionArrayResize -) - generate_function_register(Projection FunctionOneOrZero FunctionProject diff --git a/dbms/src/Functions/EmptyImpl.h b/dbms/src/Functions/EmptyImpl.h new file mode 100644 index 00000000000..f42e1331bd4 --- /dev/null +++ b/dbms/src/Functions/EmptyImpl.h @@ -0,0 +1,59 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +template +struct EmptyImpl +{ + /// If the function will return constant value for FixedString data type. + static constexpr auto is_fixed_to_constant = false; + + static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + ColumnString::Offset prev_offset = 1; + for (size_t i = 0; i < size; ++i) + { + res[i] = negative ^ (offsets[i] == prev_offset); + prev_offset = offsets[i] + 1; + } + } + + /// Only make sense if is_fixed_to_constant. + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/) + { + throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR); + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + { + std::vector empty_chars(n); + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n)); + } + + static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = negative ^ (offsets[i] == prev_offset); + prev_offset = offsets[i]; + } + } +}; + +} diff --git a/dbms/src/Functions/FunctionHelpers.h b/dbms/src/Functions/FunctionHelpers.h index 823d26f4a4c..88737ca86d0 100644 --- a/dbms/src/Functions/FunctionHelpers.h +++ b/dbms/src/Functions/FunctionHelpers.h @@ -20,13 +20,6 @@ const Type * checkAndGetDataType(const IDataType * data_type) return typeid_cast(data_type); } -template -bool checkDataType(const IDataType * data_type) -{ - return checkAndGetDataType(data_type); -} - - template const Type * checkAndGetColumn(const IColumn * column) { diff --git a/dbms/src/Functions/FunctionNumericPredicate.h b/dbms/src/Functions/FunctionNumericPredicate.h new file mode 100644 index 00000000000..ed26d3c5ee0 --- /dev/null +++ b/dbms/src/Functions/FunctionNumericPredicate.h @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionNumericPredicate : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(arguments.front())) + throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto in = block.getByPosition(arguments.front()).column.get(); + + if ( !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result)) + throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + + template + bool execute(Block & block, const IColumn * in_untyped, const size_t result) + { + if (const auto in = checkAndGetColumn>(in_untyped)) + { + const auto size = in->size(); + + auto out = ColumnUInt8::create(size); + + const auto & in_data = in->getData(); + auto & out_data = out->getData(); + + for (const auto i : ext::range(0, size)) + out_data[i] = Impl::execute(in_data[i]); + + block.getByPosition(result).column = std::move(out); + return true; + } + + return false; + } +}; + +} diff --git a/dbms/src/Functions/FunctionStartsEndsWith.h b/dbms/src/Functions/FunctionStartsEndsWith.h new file mode 100644 index 00000000000..4742fc47e29 --- /dev/null +++ b/dbms/src/Functions/FunctionStartsEndsWith.h @@ -0,0 +1,139 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct NameStartsWith +{ + static constexpr auto name = "startsWith"; +}; +struct NameEndsWith +{ + static constexpr auto name = "endsWith"; +}; + +template +class FunctionStartsEndsWith : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); + const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + vec_res.resize(input_rows_count); + + if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) + dispatch(StringSource(*haystack), needle_column, vec_res); + else if (const ColumnFixedString * haystack = checkAndGetColumn(haystack_column)) + dispatch(FixedStringSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) + dispatch>(ConstSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) + dispatch>(ConstSource(*haystack), needle_column, vec_res); + else + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + block.getByPosition(result).column = std::move(col_res); + } + +private: + template + void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + { + if (const ColumnString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, StringSource(*needle), res_data); + else if (const ColumnFixedString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, FixedStringSource(*needle), res_data); + else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle), res_data); + else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle), res_data); + else + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + template + static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) + { + size_t row_num = 0; + + while (!haystack_source.isEnd()) + { + auto haystack = haystack_source.getWhole(); + auto needle = needle_source.getWhole(); + + if (needle.size > haystack.size) + { + res_data[row_num] = false; + } + else + { + if constexpr (std::is_same_v) + { + res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); + } + else /// endsWith + { + res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); + } + } + + haystack_source.next(); + needle_source.next(); + ++row_num; + } + } +}; + +} diff --git a/dbms/src/Functions/FunctionStringOrArrayToT.h b/dbms/src/Functions/FunctionStringOrArrayToT.h new file mode 100644 index 00000000000..88f24630862 --- /dev/null +++ b/dbms/src/Functions/FunctionStringOrArrayToT.h @@ -0,0 +1,101 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringOrArrayToT : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0]) + && !isArray(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared>(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::vector(col->getChars(), col->getOffsets(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + if (Impl::is_fixed_to_constant) + { + ResultType res = 0; + Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res); + + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res)); + } + else + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + } + else if (const ColumnArray * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::array(col->getOffsets(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/FunctionStringToString.h b/dbms/src/Functions/FunctionStringToString.h new file mode 100644 index 00000000000..65d505cd2a5 --- /dev/null +++ b/dbms/src/Functions/FunctionStringToString.h @@ -0,0 +1,76 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringToString : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isInjective(const Block &) override + { + return is_injective; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnFixedString::create(col->getN()); + Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 7132d342c0e..d8d27e5ce23 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; extern const int DECIMAL_OVERFLOW; + extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; } @@ -1130,6 +1131,123 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name.str(), context); } + bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const + { + if constexpr (!std::is_same_v, MultiplyImpl>) + return false; + + WhichDataType which0(type0); + WhichDataType which1(type1); + + return (which0.isAggregateFunction() && which1.isNativeUInt()) + || (which0.isNativeUInt() && which1.isAggregateFunction()); + } + + bool isAggregateAddition(const DataTypePtr & type0, const DataTypePtr & type1) const + { + if constexpr (!std::is_same_v, PlusImpl>) + return false; + + WhichDataType which0(type0); + WhichDataType which1(type1); + + return which0.isAggregateFunction() && which1.isAggregateFunction(); + } + + /// Multiply aggregation state by integer constant: by merging it with itself specified number of times. + void executeAggregateMultiply(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const + { + ColumnNumbers new_arguments = arguments; + if (WhichDataType(block.getByPosition(new_arguments[1]).type).isAggregateFunction()) + std::swap(new_arguments[0], new_arguments[1]); + + if (!block.getByPosition(new_arguments[1]).column->isColumnConst()) + throw Exception{"Illegal column " + block.getByPosition(new_arguments[1]).column->getName() + + " of argument of aggregation state multiply. Should be integer constant", ErrorCodes::ILLEGAL_COLUMN}; + + const ColumnAggregateFunction * column = typeid_cast(block.getByPosition(new_arguments[0]).column.get()); + IAggregateFunction * function = column->getAggregateFunction().get(); + + auto arena = std::make_shared(); + + auto column_to = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); + column_to->reserve(input_rows_count); + + auto column_from = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); + column_from->reserve(input_rows_count); + + for (size_t i = 0; i < input_rows_count; ++i) + { + column_to->insertDefault(); + column_from->insertFrom(column->getData()[i]); + } + + auto & vec_to = column_to->getData(); + auto & vec_from = column_from->getData(); + + UInt64 m = typeid_cast(block.getByPosition(new_arguments[1]).column.get())->getValue(); + + /// We use exponentiation by squaring algorithm to perform multiplying aggregate states by N in O(log(N)) operations + /// https://en.wikipedia.org/wiki/Exponentiation_by_squaring + while (m) + { + if (m % 2) + { + for (size_t i = 0; i < input_rows_count; ++i) + function->merge(vec_to[i], vec_from[i], arena.get()); + --m; + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + function->merge(vec_from[i], vec_from[i], arena.get()); + m /= 2; + } + } + + block.getByPosition(result).column = std::move(column_to); + } + + /// Merge two aggregation states together. + void executeAggregateAddition(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const + { + const ColumnAggregateFunction * columns[2]; + for (size_t i = 0; i < 2; ++i) + columns[i] = typeid_cast(block.getByPosition(arguments[i]).column.get()); + + auto column_to = ColumnAggregateFunction::create(columns[0]->getAggregateFunction()); + column_to->reserve(input_rows_count); + + for(size_t i = 0; i < input_rows_count; ++i) + { + column_to->insertFrom(columns[0]->getData()[i]); + column_to->insertMergeFrom(columns[1]->getData()[i]); + } + + block.getByPosition(result).column = std::move(column_to); + } + + void executeDateTimeIntervalPlusMinus(Block & block, const ColumnNumbers & arguments, + size_t result, size_t input_rows_count, const FunctionBuilderPtr & function_builder) const + { + ColumnNumbers new_arguments = arguments; + + /// Interval argument must be second. + if (WhichDataType(block.getByPosition(arguments[0]).type).isInterval()) + std::swap(new_arguments[0], new_arguments[1]); + + /// Change interval argument type to its representation + Block new_block = block; + new_block.getByPosition(new_arguments[1]).type = std::make_shared>(); + + ColumnsWithTypeAndName new_arguments_with_type_and_name = + {new_block.getByPosition(new_arguments[0]), new_block.getByPosition(new_arguments[1])}; + auto function = function_builder->build(new_arguments_with_type_and_name); + + function->execute(new_block, new_arguments, result, input_rows_count); + block.getByPosition(result).column = new_block.getByPosition(result).column; + } + public: static constexpr auto name = Name::name; static FunctionPtr create(const Context & context) { return std::make_shared(context); } @@ -1151,11 +1269,21 @@ public: /// Special case when multiply aggregate function state if (isAggregateMultiply(arguments[0], arguments[1])) { - if (checkDataType(arguments[0].get())) + if (WhichDataType(arguments[0]).isAggregateFunction()) return arguments[0]; return arguments[1]; } + /// Special case - addition of two aggregate functions states + if (isAggregateAddition(arguments[0], arguments[1])) + { + if (!arguments[0]->equals(*arguments[1])) + throw Exception("Cannot add aggregate states of different functions: " + + arguments[0]->getName() + " and " + arguments[1]->getName(), ErrorCodes::CANNOT_ADD_DIFFERENT_AGGREGATE_STATES); + + return arguments[0]; + } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1])) { @@ -1165,7 +1293,7 @@ public: new_arguments[i].type = arguments[i]; /// Interval argument must be second. - if (checkDataType(new_arguments[0].type.get())) + if (WhichDataType(new_arguments[0].type).isInterval()) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument to its representation @@ -1206,92 +1334,26 @@ public: return type_res; } - bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const - { - if constexpr (!std::is_same_v, MultiplyImpl>) - return false; - auto is_uint_type = [](const DataTypePtr & type) - { - return checkDataType(type.get()) || checkDataType(type.get()) - || checkDataType(type.get()) || checkDataType(type.get()); - }; - return ((checkDataType(type0.get()) && is_uint_type(type1)) - || (is_uint_type(type0) && checkDataType(type1.get()))); - } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { /// Special case when multiply aggregate function state if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) { - ColumnNumbers new_arguments = arguments; - if (checkDataType(block.getByPosition(new_arguments[1]).type.get())) - std::swap(new_arguments[0], new_arguments[1]); + executeAggregateMultiply(block, arguments, result, input_rows_count); + return; + } - const ColumnAggregateFunction * column = typeid_cast(block.getByPosition(new_arguments[0]).column.get()); - IAggregateFunction * function = column->getAggregateFunction().get(); - - auto arena = std::make_shared(); - - auto column_to = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); - column_to->reserve(input_rows_count); - - auto column_from = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); - column_from->reserve(input_rows_count); - - for (size_t i = 0; i < input_rows_count; ++i) - { - column_to->insertDefault(); - column_from->insertFrom(column->getData()[i]); - } - - auto & vec_to = column_to->getData(); - auto & vec_from = column_from->getData(); - - UInt64 m = block.getByPosition(new_arguments[1]).column->getUInt(0); - - /// We use exponentiation by squaring algorithm to perform multiplying aggregate states by N in O(log(N)) operations - /// https://en.wikipedia.org/wiki/Exponentiation_by_squaring - while (m) - { - if (m % 2) - { - for (size_t i = 0; i < input_rows_count; ++i) - function->merge(vec_to[i], vec_from[i], arena.get()); - --m; - } - else - { - for (size_t i = 0; i < input_rows_count; ++i) - function->merge(vec_from[i], vec_from[i], arena.get()); - m /= 2; - } - } - - block.getByPosition(result).column = std::move(column_to); + /// Special case - addition of two aggregate functions states + if (isAggregateAddition(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) + { + executeAggregateAddition(block, arguments, result, input_rows_count); return; } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) { - ColumnNumbers new_arguments = arguments; - - /// Interval argument must be second. - if (checkDataType(block.getByPosition(arguments[0]).type.get())) - std::swap(new_arguments[0], new_arguments[1]); - - /// Change interval argument type to its representation - Block new_block = block; - new_block.getByPosition(new_arguments[1]).type = std::make_shared>(); - - ColumnsWithTypeAndName new_arguments_with_type_and_name = - {new_block.getByPosition(new_arguments[0]), new_block.getByPosition(new_arguments[1])}; - auto function = function_builder->build(new_arguments_with_type_and_name); - - function->execute(new_block, new_arguments, result, input_rows_count); - block.getByPosition(result).column = new_block.getByPosition(result).column; - + executeDateTimeIntervalPlusMinus(block, arguments, result, input_rows_count, function_builder); return; } @@ -1906,17 +1968,17 @@ public: throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be at least 2.", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; - const auto first_arg = arguments.front().get(); + const auto & first_arg = arguments.front(); - if (!first_arg->isInteger()) + if (!isInteger(first_arg)) throw Exception{"Illegal type " + first_arg->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; for (const auto i : ext::range(1, arguments.size())) { - const auto pos_arg = arguments[i].get(); + const auto & pos_arg = arguments[i]; - if (!pos_arg->isUnsignedInteger()) + if (!isUnsignedInteger(pos_arg)) throw Exception{"Illegal type " + pos_arg->getName() + " of " + toString(i) + " argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } diff --git a/dbms/src/Functions/FunctionsArray.cpp b/dbms/src/Functions/FunctionsArray.cpp deleted file mode 100644 index 02257437e7e..00000000000 --- a/dbms/src/Functions/FunctionsArray.cpp +++ /dev/null @@ -1,3791 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ARGUMENT_OUT_OF_BOUND; -} - - -/// Implementation of FunctionArray. - -String FunctionArray::getName() const -{ - return name; -} - -FunctionPtr FunctionArray::create(const Context & context) -{ - return std::make_shared(context); -} - -FunctionArray::FunctionArray(const Context & context) - : context(context) -{ -} - -DataTypePtr FunctionArray::getReturnTypeImpl(const DataTypes & arguments) const -{ - return std::make_shared(getLeastSupertype(arguments)); -} - -void FunctionArray::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - size_t num_elements = arguments.size(); - - if (num_elements == 0) - { - /// We should return constant empty array. - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - const DataTypePtr & return_type = block.getByPosition(result).type; - const DataTypePtr & elem_type = static_cast(*return_type).getNestedType(); - - size_t block_size = input_rows_count; - - /** If part of columns have not same type as common type of all elements of array, - * then convert them to common type. - * If part of columns are constants, - * then convert them to full columns. - */ - - Columns columns_holder(num_elements); - const IColumn * columns[num_elements]; - - for (size_t i = 0; i < num_elements; ++i) - { - const auto & arg = block.getByPosition(arguments[i]); - - ColumnPtr preprocessed_column = arg.column; - - if (!arg.type->equals(*elem_type)) - preprocessed_column = castColumn(arg, elem_type, context); - - if (ColumnPtr materialized_column = preprocessed_column->convertToFullColumnIfConst()) - preprocessed_column = materialized_column; - - columns_holder[i] = std::move(preprocessed_column); - columns[i] = columns_holder[i].get(); - } - - /// Create and fill the result array. - - auto out = ColumnArray::create(elem_type->createColumn()); - IColumn & out_data = out->getData(); - IColumn::Offsets & out_offsets = out->getOffsets(); - - out_data.reserve(block_size * num_elements); - out_offsets.resize(block_size); - - IColumn::Offset current_offset = 0; - for (size_t i = 0; i < block_size; ++i) - { - for (size_t j = 0; j < num_elements; ++j) - out_data.insertFrom(*columns[j], i); - - current_offset += num_elements; - out_offsets[i] = current_offset; - } - - block.getByPosition(result).column = std::move(out); -} - - -/// Implementation of FunctionArrayElement. - -namespace ArrayImpl -{ - -class NullMapBuilder -{ -public: - operator bool() const { return src_null_map; } - bool operator!() const { return !src_null_map; } - - void initSource(const UInt8 * src_null_map_) - { - src_null_map = src_null_map_; - } - - void initSink(size_t size) - { - auto sink = ColumnUInt8::create(size); - sink_null_map = sink->getData().data(); - sink_null_map_holder = std::move(sink); - } - - void update(size_t from) - { - sink_null_map[index] = bool(src_null_map && src_null_map[from]); - ++index; - } - - void update() - { - sink_null_map[index] = bool(src_null_map); - ++index; - } - - ColumnPtr getNullMapColumnPtr() && { return std::move(sink_null_map_holder); } - -private: - const UInt8 * src_null_map = nullptr; - UInt8 * sink_null_map = nullptr; - MutableColumnPtr sink_null_map_holder; - size_t index = 0; -}; - -} - -namespace -{ - -template -struct ArrayElementNumImpl -{ - /** Implementation for constant index. - * If negative = false - index is from beginning of array, started from 0. - * If negative = true - index is from end of array, started from 0. - */ - template - static void vectorConst( - const PaddedPODArray & data, const ColumnArray::Offsets & offsets, - const ColumnArray::Offset index, - PaddedPODArray & result, ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result.resize(size); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - - if (index < array_size) - { - size_t j = !negative ? (current_offset + index) : (offsets[i] - index - 1); - result[i] = data[j]; - if (builder) - builder.update(j); - } - else - { - result[i] = T(); - - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } - - /** Implementation for non-constant index. - */ - template - static void vector( - const PaddedPODArray & data, const ColumnArray::Offsets & offsets, - const PaddedPODArray & indices, - PaddedPODArray & result, ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result.resize(size); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - - TIndex index = indices[i]; - if (index > 0 && static_cast(index) <= array_size) - { - size_t j = current_offset + index - 1; - result[i] = data[j]; - - if (builder) - builder.update(j); - } - else if (index < 0 && static_cast(-index) <= array_size) - { - size_t j = offsets[i] + index; - result[i] = data[j]; - - if (builder) - builder.update(j); - } - else - { - result[i] = T(); - - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } -}; - -struct ArrayElementStringImpl -{ - template - static void vectorConst( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, - const ColumnArray::Offset index, - ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, - ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result_offsets.resize(size); - result_data.reserve(data.size()); - - ColumnArray::Offset current_offset = 0; - ColumnArray::Offset current_result_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - - if (index < array_size) - { - size_t adjusted_index = !negative ? index : (array_size - index - 1); - - size_t j = current_offset + adjusted_index; - if (builder) - builder.update(j); - - ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0 - ? 0 - : string_offsets[current_offset + adjusted_index - 1]; - - ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos; - - result_data.resize(current_result_offset + string_size); - memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size); - current_result_offset += string_size; - result_offsets[i] = current_result_offset; - } - else - { - /// Insert an empty row. - result_data.resize(current_result_offset + 1); - result_data[current_result_offset] = 0; - current_result_offset += 1; - result_offsets[i] = current_result_offset; - - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } - - /** Implementation for non-constant index. - */ - template - static void vector( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, - const PaddedPODArray & indices, - ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, - ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result_offsets.resize(size); - result_data.reserve(data.size()); - - ColumnArray::Offset current_offset = 0; - ColumnArray::Offset current_result_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - size_t adjusted_index; /// index in array from zero - - TIndex index = indices[i]; - if (index > 0 && static_cast(index) <= array_size) - adjusted_index = index - 1; - else if (index < 0 && static_cast(-index) <= array_size) - adjusted_index = array_size + index; - else - adjusted_index = array_size; /// means no element should be taken - - if (adjusted_index < array_size) - { - size_t j = current_offset + adjusted_index; - if (builder) - builder.update(j); - - ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0 - ? 0 - : string_offsets[current_offset + adjusted_index - 1]; - - ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos; - - result_data.resize(current_result_offset + string_size); - memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size); - current_result_offset += string_size; - result_offsets[i] = current_result_offset; - } - else - { - /// Insert empty string - result_data.resize(current_result_offset + 1); - result_data[current_result_offset] = 0; - current_result_offset += 1; - result_offsets[i] = current_result_offset; - - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } -}; - -/// Generic implementation for other nested types. -struct ArrayElementGenericImpl -{ - template - static void vectorConst( - const IColumn & data, const ColumnArray::Offsets & offsets, - const ColumnArray::Offset index, - IColumn & result, ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result.reserve(size); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - - if (index < array_size) - { - size_t j = !negative ? current_offset + index : offsets[i] - index - 1; - result.insertFrom(data, j); - if (builder) - builder.update(j); - } - else - { - result.insertDefault(); - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } - - /** Implementation for non-constant index. - */ - template - static void vector( - const IColumn & data, const ColumnArray::Offsets & offsets, - const PaddedPODArray & indices, - IColumn & result, ArrayImpl::NullMapBuilder & builder) - { - size_t size = offsets.size(); - result.reserve(size); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - size_t array_size = offsets[i] - current_offset; - - TIndex index = indices[i]; - if (index > 0 && static_cast(index) <= array_size) - { - size_t j = current_offset + index - 1; - result.insertFrom(data, j); - if (builder) - builder.update(j); - } - else if (index < 0 && static_cast(-index) <= array_size) - { - size_t j = offsets[i] + index; - result.insertFrom(data, j); - if (builder) - builder.update(j); - } - else - { - result.insertDefault(); - if (builder) - builder.update(); - } - - current_offset = offsets[i]; - } - } -}; - -} - - -FunctionPtr FunctionArrayElement::create(const Context &) -{ - return std::make_shared(); -} - - -template -bool FunctionArrayElement::executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const ColumnVector * col_nested = checkAndGetColumn>(&col_array->getData()); - - if (!col_nested) - return false; - - auto col_res = ColumnVector::create(); - - if (index.getType() == Field::Types::UInt64) - ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), safeGet(index) - 1, col_res->getData(), builder); - else if (index.getType() == Field::Types::Int64) - ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), -safeGet(index) - 1, col_res->getData(), builder); - else - throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -template -bool FunctionArrayElement::executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const ColumnVector * col_nested = checkAndGetColumn>(&col_array->getData()); - - if (!col_nested) - return false; - - auto col_res = ColumnVector::create(); - - ArrayElementNumImpl::template vector( - col_nested->getData(), col_array->getOffsets(), indices, col_res->getData(), builder); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -bool FunctionArrayElement::executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const ColumnString * col_nested = checkAndGetColumn(&col_array->getData()); - - if (!col_nested) - return false; - - auto col_res = ColumnString::create(); - - if (index.getType() == Field::Types::UInt64) - ArrayElementStringImpl::vectorConst( - col_nested->getChars(), - col_array->getOffsets(), - col_nested->getOffsets(), - safeGet(index) - 1, - col_res->getChars(), - col_res->getOffsets(), - builder); - else if (index.getType() == Field::Types::Int64) - ArrayElementStringImpl::vectorConst( - col_nested->getChars(), - col_array->getOffsets(), - col_nested->getOffsets(), - -safeGet(index) - 1, - col_res->getChars(), - col_res->getOffsets(), - builder); - else - throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -template -bool FunctionArrayElement::executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const ColumnString * col_nested = checkAndGetColumn(&col_array->getData()); - - if (!col_nested) - return false; - - auto col_res = ColumnString::create(); - - ArrayElementStringImpl::vector( - col_nested->getChars(), - col_array->getOffsets(), - col_nested->getOffsets(), - indices, - col_res->getChars(), - col_res->getOffsets(), - builder); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -bool FunctionArrayElement::executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const auto & col_nested = col_array->getData(); - auto col_res = col_nested.cloneEmpty(); - - if (index.getType() == Field::Types::UInt64) - ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), safeGet(index) - 1, *col_res, builder); - else if (index.getType() == Field::Types::Int64) - ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), -safeGet(index) - 1, *col_res, builder); - else - throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -template -bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder) -{ - const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const auto & col_nested = col_array->getData(); - auto col_res = col_nested.cloneEmpty(); - - ArrayElementGenericImpl::vector( - col_nested, col_array->getOffsets(), indices, *col_res, builder); - - block.getByPosition(result).column = std::move(col_res); - return true; -} - -template -bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder, - size_t input_rows_count) -{ - const ColumnArray * col_array = checkAndGetColumnConstData(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - auto res = block.getByPosition(result).type->createColumn(); - - size_t rows = input_rows_count; - const IColumn & array_elements = col_array->getData(); - size_t array_size = array_elements.size(); - - for (size_t i = 0; i < rows; ++i) - { - IndexType index = indices[i]; - if (index > 0 && static_cast(index) <= array_size) - { - size_t j = index - 1; - res->insertFrom(array_elements, j); - if (builder) - builder.update(j); - } - else if (index < 0 && static_cast(-index) <= array_size) - { - size_t j = array_size + index; - res->insertFrom(array_elements, j); - if (builder) - builder.update(j); - } - else - { - res->insertDefault(); - if (builder) - builder.update(); - } - } - - block.getByPosition(result).column = std::move(res); - return true; -} - -template -bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, - ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) -{ - auto index = checkAndGetColumn>(block.getByPosition(arguments[1]).column.get()); - - if (!index) - return false; - - const auto & index_data = index->getData(); - - if (builder) - builder.initSink(index_data.size()); - - if (!( executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeNumber(block, arguments, result, index_data, builder) - || executeConst(block, arguments, result, index_data, builder, input_rows_count) - || executeString(block, arguments, result, index_data, builder) - || executeGeneric(block, arguments, result, index_data, builder))) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - - return true; -} - -bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const ColumnArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); - - if (!col_array) - return false; - - const ColumnTuple * col_nested = typeid_cast(&col_array->getData()); - - if (!col_nested) - return false; - - const Columns & tuple_columns = col_nested->getColumns(); - size_t tuple_size = tuple_columns.size(); - - const DataTypes & tuple_types = typeid_cast( - *typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType()).getElements(); - - /** We will calculate the function for the tuple of the internals of the array. - * To do this, create a temporary block. - * It will consist of the following columns - * - the index of the array to be taken; - * - an array of the first elements of the tuples; - * - the result of taking the elements by the index for an array of the first elements of the tuples; - * - array of the second elements of the tuples; - * - result of taking elements by index for an array of second elements of tuples; - * ... - */ - Block block_of_temporary_results; - block_of_temporary_results.insert(block.getByPosition(arguments[1])); - - /// results of taking elements by index for arrays from each element of the tuples; - Columns result_tuple_columns; - - for (size_t i = 0; i < tuple_size; ++i) - { - ColumnWithTypeAndName array_of_tuple_section; - array_of_tuple_section.column = ColumnArray::create(tuple_columns[i], col_array->getOffsetsPtr()); - array_of_tuple_section.type = std::make_shared(tuple_types[i]); - block_of_temporary_results.insert(array_of_tuple_section); - - ColumnWithTypeAndName array_elements_of_tuple_section; - array_elements_of_tuple_section.type = getReturnTypeImpl( - {block_of_temporary_results.getByPosition(i * 2 + 1).type, block_of_temporary_results.getByPosition(0).type}); - block_of_temporary_results.insert(array_elements_of_tuple_section); - - executeImpl(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2, input_rows_count); - - result_tuple_columns.emplace_back(std::move(block_of_temporary_results.getByPosition(i * 2 + 2).column)); - } - - block.getByPosition(result).column = ColumnTuple::create(result_tuple_columns); - - return true; -} - -String FunctionArrayElement::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!arguments[1]->isInteger()) - throw Exception("Second argument for function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return array_type->getNestedType(); -} - -void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - /// Check nullability. - bool is_array_of_nullable = false; - - const ColumnArray * col_array = nullptr; - const ColumnArray * col_const_array = nullptr; - - col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - if (col_array) - is_array_of_nullable = col_array->getData().isColumnNullable(); - else - { - col_const_array = checkAndGetColumnConstData(block.getByPosition(arguments[0]).column.get()); - if (col_const_array) - is_array_of_nullable = col_const_array->getData().isColumnNullable(); - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - } - - if (!is_array_of_nullable) - { - ArrayImpl::NullMapBuilder builder; - perform(block, arguments, result, builder, input_rows_count); - } - else - { - /// Perform initializations. - ArrayImpl::NullMapBuilder builder; - Block source_block; - - const auto & input_type = typeid_cast(*typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType(); - const auto & tmp_ret_type = typeid_cast(*block.getByPosition(result).type).getNestedType(); - - if (col_array) - { - const auto & nullable_col = typeid_cast(col_array->getData()); - const auto & nested_col = nullable_col.getNestedColumnPtr(); - - /// Put nested_col inside a ColumnArray. - source_block = - { - { - ColumnArray::create(nested_col, col_array->getOffsetsPtr()), - std::make_shared(input_type), - "" - }, - block.getByPosition(arguments[1]), - { - nullptr, - tmp_ret_type, - "" - } - }; - - builder.initSource(nullable_col.getNullMapData().data()); - } - else - { - /// ColumnConst(ColumnArray(ColumnNullable(...))) - const auto & nullable_col = static_cast(col_const_array->getData()); - const auto & nested_col = nullable_col.getNestedColumnPtr(); - - source_block = - { - { - ColumnConst::create(ColumnArray::create(nested_col, col_const_array->getOffsetsPtr()), input_rows_count), - std::make_shared(input_type), - "" - }, - block.getByPosition(arguments[1]), - { - nullptr, - tmp_ret_type, - "" - } - }; - - builder.initSource(nullable_col.getNullMapData().data()); - } - - perform(source_block, {0, 1}, 2, builder, input_rows_count); - - /// Store the result. - const ColumnWithTypeAndName & source_col = source_block.getByPosition(2); - ColumnWithTypeAndName & dest_col = block.getByPosition(result); - dest_col.column = ColumnNullable::create(source_col.column, builder ? std::move(builder).getNullMapColumnPtr() : ColumnUInt8::create()); - } -} - -void FunctionArrayElement::perform(Block & block, const ColumnNumbers & arguments, size_t result, - ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) -{ - if (executeTuple(block, arguments, result, input_rows_count)) - { - } - else if (!block.getByPosition(arguments[1]).column->isColumnConst()) - { - if (!(executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count) - || executeArgument(block, arguments, result, builder, input_rows_count))) - throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.", - ErrorCodes::ILLEGAL_COLUMN); - } - else - { - Field index = (*block.getByPosition(arguments[1]).column)[0]; - - if (builder) - builder.initSink(input_rows_count); - - if (index == UInt64(0)) - throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - - if (!( executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeNumberConst(block, arguments, result, index, builder) - || executeStringConst (block, arguments, result, index, builder) - || executeGenericConst (block, arguments, result, index, builder))) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -} - -/// Implementation of FunctionArrayEnumerate. - -FunctionPtr FunctionArrayEnumerate::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionArrayEnumerate::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayEnumerate::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be an array but it has type " - + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(std::make_shared()); -} - -void FunctionArrayEnumerate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - if (const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) - { - const ColumnArray::Offsets & offsets = array->getOffsets(); - - auto res_nested = ColumnUInt32::create(); - - ColumnUInt32::Container & res_values = res_nested->getData(); - res_values.resize(array->getData().size()); - ColumnArray::Offset prev_off = 0; - for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) - { - ColumnArray::Offset off = offsets[i]; - for (ColumnArray::Offset j = prev_off; j < off; ++j) - res_values[j] = j - prev_off + 1; - - prev_off = off; - } - - block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr()); - } - else - { - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -} - -/// Implementation of FunctionArrayUniq. - -FunctionPtr FunctionArrayUniq::create(const Context &) { return std::make_shared(); } - -String FunctionArrayUniq::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayUniq::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments.size() == 0) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be at least 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - for (size_t i = 0; i < arguments.size(); ++i) - { - const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); - if (!array_type) - throw Exception("All arguments for function " + getName() + " must be arrays but argument " + - toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return std::make_shared(); -} - -void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - Columns array_columns(arguments.size()); - const ColumnArray::Offsets * offsets = nullptr; - ColumnRawPtrs data_columns(arguments.size()); - ColumnRawPtrs original_data_columns(arguments.size()); - ColumnRawPtrs null_maps(arguments.size()); - - bool has_nullable_columns = false; - - for (size_t i = 0; i < arguments.size(); ++i) - { - ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); - if (!array) - { - const ColumnConst * const_array = checkAndGetColumnConst( - block.getByPosition(arguments[i]).column.get()); - if (!const_array) - throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() - + " of " + toString(i + 1) + getOrdinalSuffix(i + 1) + " argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - array_ptr = const_array->convertToFullColumn(); - array = static_cast(array_ptr.get()); - } - - array_columns[i] = array_ptr; - - const ColumnArray::Offsets & offsets_i = array->getOffsets(); - if (i == 0) - offsets = &offsets_i; - else if (offsets_i != *offsets) - throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", - ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - - data_columns[i] = &array->getData(); - original_data_columns[i] = data_columns[i]; - - if (data_columns[i]->isColumnNullable()) - { - has_nullable_columns = true; - const auto & nullable_col = static_cast(*data_columns[i]); - data_columns[i] = &nullable_col.getNestedColumn(); - null_maps[i] = &nullable_col.getNullMapColumn(); - } - else - null_maps[i] = nullptr; - } - - const ColumnArray * first_array = static_cast(array_columns[0].get()); - const IColumn * first_null_map = null_maps[0]; - auto res = ColumnUInt32::create(); - - ColumnUInt32::Container & res_values = res->getData(); - res_values.resize(offsets->size()); - - if (arguments.size() == 1) - { - if (!( executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeString(first_array, first_null_map, res_values))) - executeHashed(*offsets, original_data_columns, res_values); - } - else - { - if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) - executeHashed(*offsets, original_data_columns, res_values); - } - - block.getByPosition(result).column = std::move(res); -} - -template -bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) -{ - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnVector * nested = checkAndGetColumn>(inner_col); - if (!nested) - return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - const typename ColumnVector::Container & values = nested->getData(); - - using Set = ClearableHashSet, HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - - Set set; - ColumnArray::Offset prev_off = 0; - for (size_t i = 0; i < offsets.size(); ++i) - { - set.clear(); - bool found_null = false; - ColumnArray::Offset off = offsets[i]; - for (ColumnArray::Offset j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - found_null = true; - else - set.insert(values[j]); - } - - res_values[i] = set.size() + found_null; - prev_off = off; - } - return true; -} - -bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) -{ - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnString * nested = checkAndGetColumn(inner_col); - if (!nested) - return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - - using Set = ClearableHashSet, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - - Set set; - ColumnArray::Offset prev_off = 0; - for (size_t i = 0; i < offsets.size(); ++i) - { - set.clear(); - bool found_null = false; - ColumnArray::Offset off = offsets[i]; - for (ColumnArray::Offset j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - found_null = true; - else - set.insert(nested->getDataAt(j)); - } - - res_values[i] = set.size() + found_null; - prev_off = off; - } - return true; -} - - -bool FunctionArrayUniq::execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns) -{ - size_t count = columns.size(); - size_t keys_bytes = 0; - Sizes key_sizes(count); - - for (size_t j = 0; j < count; ++j) - { - if (!columns[j]->isFixedAndContiguous()) - return false; - key_sizes[j] = columns[j]->sizeOfValueIfFixed(); - keys_bytes += key_sizes[j]; - } - if (has_nullable_columns) - keys_bytes += std::tuple_size>::value; - - if (keys_bytes > 16) - return false; - - using Set = ClearableHashSet, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - /// Suppose that, for a given row, each of the N columns has an array whose length is M. - /// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed: - /// - /// col1 ... colN - /// - /// arr_1[1], ..., arr_N[1] -> pack into a binary blob b1 - /// . - /// . - /// . - /// arr_1[M], ..., arr_N[M] -> pack into a binary blob bM - /// - /// Each binary blob is inserted into a hash table. - /// - Set set; - ColumnArray::Offset prev_off = 0; - for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) - { - set.clear(); - ColumnArray::Offset off = offsets[i]; - for (ColumnArray::Offset j = prev_off; j < off; ++j) - { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (ColumnArray::Offset i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - ColumnArray::Offset bucket = i / 8; - ColumnArray::Offset offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - set.insert(packFixed(j, count, columns, key_sizes, bitmap)); - } - else - set.insert(packFixed(j, count, columns, key_sizes)); - } - - res_values[i] = set.size(); - prev_off = off; - } - - return true; -} - -void FunctionArrayUniq::executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values) -{ - size_t count = columns.size(); - - using Set = ClearableHashSet, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - Set set; - ColumnArray::Offset prev_off = 0; - for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) - { - set.clear(); - ColumnArray::Offset off = offsets[i]; - for (ColumnArray::Offset j = prev_off; j < off; ++j) - set.insert(hash128(j, count, columns)); - - res_values[i] = set.size(); - prev_off = off; - } -} - -/// Implementation of FunctionArrayDistinct. - -FunctionPtr FunctionArrayDistinct::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionArrayDistinct::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayDistinct::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception("Argument for function " + getName() + " must be array but it " - " has type " + arguments[0]->getName() + ".", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto nested_type = removeNullable(array_type->getNestedType()); - - return std::make_shared(nested_type); -} - -void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - ColumnPtr array_ptr = block.getByPosition(arguments[0]).column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); - - const auto & return_type = block.getByPosition(result).type; - - auto res_ptr = return_type->createColumn(); - ColumnArray & res = static_cast(*res_ptr); - - const IColumn & src_data = array->getData(); - const ColumnArray::Offsets & offsets = array->getOffsets(); - - IColumn & res_data = res.getData(); - ColumnArray::Offsets & res_offsets = res.getOffsets(); - - const ColumnNullable * nullable_col = nullptr; - - const IColumn * inner_col; - - if (src_data.isColumnNullable()) - { - nullable_col = static_cast(&src_data); - inner_col = &nullable_col->getNestedColumn(); - } - else - { - inner_col = &src_data; - } - - if (!(executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) - || executeString(*inner_col, offsets, res_data, res_offsets, nullable_col))) - executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col); - - block.getByPosition(result).column = std::move(res_ptr); -} - -template -bool FunctionArrayDistinct::executeNumber( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col) -{ - const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data); - - if (!src_data_concrete) - { - return false; - } - - const PaddedPODArray & values = src_data_concrete->getData(); - PaddedPODArray & res_data = typeid_cast &>(res_data_col).getData(); - - const PaddedPODArray * src_null_map = nullptr; - - if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); - - using Set = ClearableHashSet, - HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - - Set set; - - ColumnArray::Offset prev_src_offset = 0; - ColumnArray::Offset res_offset = 0; - - for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) - { - set.clear(); - - ColumnArray::Offset curr_src_offset = src_offsets[i]; - for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) - { - if (nullable_col && (*src_null_map)[j]) - continue; - - if (set.find(values[j]) == set.end()) - { - res_data.emplace_back(values[j]); - set.insert(values[j]); - } - } - - res_offset += set.size(); - res_offsets.emplace_back(res_offset); - - prev_src_offset = curr_src_offset; - } - return true; -} - -bool FunctionArrayDistinct::executeString( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col) -{ - const ColumnString * src_data_concrete = checkAndGetColumn(&src_data); - - if (!src_data_concrete) - return false; - - ColumnString & res_data_column_string = typeid_cast(res_data_col); - - using Set = ClearableHashSet, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - - const PaddedPODArray * src_null_map = nullptr; - - if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); - - Set set; - - ColumnArray::Offset prev_src_offset = 0; - ColumnArray::Offset res_offset = 0; - - for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) - { - set.clear(); - - ColumnArray::Offset curr_src_offset = src_offsets[i]; - for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) - { - if (nullable_col && (*src_null_map)[j]) - continue; - - StringRef str_ref = src_data_concrete->getDataAt(j); - - if (set.find(str_ref) == set.end()) - { - set.insert(str_ref); - res_data_column_string.insertData(str_ref.data, str_ref.size); - } - } - - res_offset += set.size(); - res_offsets.emplace_back(res_offset); - - prev_src_offset = curr_src_offset; - } - return true; -} - -void FunctionArrayDistinct::executeHashed( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col) -{ - using Set = ClearableHashSet, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - const PaddedPODArray * src_null_map = nullptr; - - if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); - - Set set; - - ColumnArray::Offset prev_src_offset = 0; - ColumnArray::Offset res_offset = 0; - - for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) - { - set.clear(); - - ColumnArray::Offset curr_src_offset = src_offsets[i]; - for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) - { - if (nullable_col && (*src_null_map)[j]) - continue; - - UInt128 hash; - SipHash hash_function; - src_data.updateHashWithValue(j, hash_function); - hash_function.get128(reinterpret_cast(&hash)); - - if (set.find(hash) == set.end()) - { - set.insert(hash); - res_data_col.insertFrom(src_data, j); - } - } - - res_offset += set.size(); - res_offsets.emplace_back(res_offset); - - prev_src_offset = curr_src_offset; - } -} - -/// Implementation of FunctionArrayEnumerateExtended. - -template -FunctionPtr FunctionArrayEnumerateExtended::create(const Context &) -{ - return std::make_shared(); -} - -template -String FunctionArrayEnumerateExtended::getName() const -{ - return Derived::name; -} - -template -DataTypePtr FunctionArrayEnumerateExtended::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments.size() == 0) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be at least 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - for (size_t i = 0; i < arguments.size(); ++i) - { - const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); - if (!array_type) - throw Exception("All arguments for function " + getName() + " must be arrays but argument " + - toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return std::make_shared(std::make_shared()); -} - -template -void FunctionArrayEnumerateExtended::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const ColumnArray::Offsets * offsets = nullptr; - ColumnRawPtrs data_columns; - data_columns.reserve(arguments.size()); - - bool has_nullable_columns = false; - - for (size_t i = 0; i < arguments.size(); ++i) - { - ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); - if (!array) - { - const ColumnConst * const_array = checkAndGetColumnConst( - block.getByPosition(arguments[i]).column.get()); - if (!const_array) - throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() - + " of " + toString(i + 1) + "-th argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - array_ptr = const_array->convertToFullColumn(); - array = checkAndGetColumn(array_ptr.get()); - } - - const ColumnArray::Offsets & offsets_i = array->getOffsets(); - if (i == 0) - offsets = &offsets_i; - else if (offsets_i != *offsets) - throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", - ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - - auto * array_data = &array->getData(); - data_columns.push_back(array_data); - } - - size_t num_columns = data_columns.size(); - ColumnRawPtrs original_data_columns(num_columns); - ColumnRawPtrs null_maps(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - original_data_columns[i] = data_columns[i]; - - if (data_columns[i]->isColumnNullable()) - { - has_nullable_columns = true; - const auto & nullable_col = static_cast(*data_columns[i]); - data_columns[i] = &nullable_col.getNestedColumn(); - null_maps[i] = &nullable_col.getNullMapColumn(); - } - else - null_maps[i] = nullptr; - } - - const ColumnArray * first_array = checkAndGetColumn(block.getByPosition(arguments.at(0)).column.get()); - const IColumn * first_null_map = null_maps[0]; - auto res_nested = ColumnUInt32::create(); - - ColumnUInt32::Container & res_values = res_nested->getData(); - if (!offsets->empty()) - res_values.resize(offsets->back()); - - if (num_columns == 1) - { - if (!( executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeString (first_array, first_null_map, res_values))) - executeHashed(*offsets, original_data_columns, res_values); - } - else - { - if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) - executeHashed(*offsets, original_data_columns, res_values); - } - - block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), first_array->getOffsetsPtr()); -} - - -template -template -bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) -{ - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnVector * nested = checkAndGetColumn>(inner_col); - if (!nested) - return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - const typename ColumnVector::Container & values = nested->getData(); - - using ValuesToIndices = ClearableHashMap, HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - - ValuesToIndices indices; - size_t prev_off = 0; - if constexpr (std::is_same_v) - { - // Unique - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - UInt32 null_count = 0; - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - res_values[j] = ++null_count; - else - res_values[j] = ++indices[values[j]]; - } - prev_off = off; - } - } - else - { - // Dense - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t rank = 0; - UInt32 null_index = 0; - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - { - if (!null_index) - null_index = ++rank; - res_values[j] = null_index; - } - else - { - auto & idx = indices[values[j]]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - } - prev_off = off; - } - } - return true; -} - -template -bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) -{ - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnString * nested = checkAndGetColumn(inner_col); - if (!nested) - return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - - size_t prev_off = 0; - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - - ValuesToIndices indices; - if constexpr (std::is_same_v) - { - // Unique - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - UInt32 null_count = 0; - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - res_values[j] = ++null_count; - else - res_values[j] = ++indices[nested->getDataAt(j)]; - } - prev_off = off; - } - } - else - { - // Dense - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t rank = 0; - UInt32 null_index = 0; - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - if (null_map_data && ((*null_map_data)[j] == 1)) - { - if (!null_index) - null_index = ++rank; - res_values[j] = null_index; - } - else - { - auto & idx = indices[nested->getDataAt(j)]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - } - prev_off = off; - } - } - return true; -} - -template -bool FunctionArrayEnumerateExtended::execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns) -{ - size_t count = columns.size(); - size_t keys_bytes = 0; - Sizes key_sizes(count); - - for (size_t j = 0; j < count; ++j) - { - if (!columns[j]->isFixedAndContiguous()) - return false; - key_sizes[j] = columns[j]->sizeOfValueIfFixed(); - keys_bytes += key_sizes[j]; - } - if (has_nullable_columns) - keys_bytes += std::tuple_size>::value; - - if (keys_bytes > 16) - return false; - - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - ValuesToIndices indices; - size_t prev_off = 0; - if constexpr (std::is_same_v) - { - // Unique - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (size_t i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - size_t bucket = i / 8; - size_t offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - res_values[j] = ++indices[packFixed(j, count, columns, key_sizes, bitmap)]; - } - else - res_values[j] = ++indices[packFixed(j, count, columns, key_sizes)]; - } - prev_off = off; - } - } - else - { - // Dense - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t off = offsets[i]; - size_t rank = 0; - for (size_t j = prev_off; j < off; ++j) - { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (size_t i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - size_t bucket = i / 8; - size_t offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - auto &idx = indices[packFixed(j, count, columns, key_sizes, bitmap)]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - else - { - auto &idx = indices[packFixed(j, count, columns, key_sizes)];; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - } - prev_off = off; - } - } - - return true; -} - -template -void FunctionArrayEnumerateExtended::executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values) -{ - size_t count = columns.size(); - - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - ValuesToIndices indices; - size_t prev_off = 0; - if constexpr (std::is_same_v) - { - // Unique - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t off = offsets[i]; - for (size_t j = prev_off; j < off; ++j) - { - res_values[j] = ++indices[hash128(j, count, columns)]; - } - prev_off = off; - } - } - else - { - // Dense - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - size_t off = offsets[i]; - size_t rank = 0; - for (size_t j = prev_off; j < off; ++j) - { - auto & idx = indices[hash128(j, count, columns)]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - prev_off = off; - } - } -} - -template class FunctionArrayEnumerateExtended; -template class FunctionArrayEnumerateExtended; - -/// Implementation of FunctionEmptyArrayToSingle. - -FunctionPtr FunctionEmptyArrayToSingle::create(const Context &) { return std::make_shared(); } - -String FunctionEmptyArrayToSingle::getName() const -{ - return name; -} - -DataTypePtr FunctionEmptyArrayToSingle::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception("Argument for function " + getName() + " must be array.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; -} - - -namespace -{ - namespace FunctionEmptyArrayToSingleImpl - { - bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) - { - if (const ColumnConst * const_array = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) - { - if (const_array->getValue().empty()) - { - auto nested_type = typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType(); - - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( - input_rows_count, - Array{nested_type->getDefault()}); - } - else - block.getByPosition(result).column = block.getByPosition(arguments[0]).column; - - return true; - } - else - return false; - } - - template - bool executeNumber( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, ColumnArray::Offsets & res_offsets, - const NullMap * src_null_map, - NullMap * res_null_map) - { - if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) - { - const PaddedPODArray & src_data = src_data_concrete->getData(); - PaddedPODArray & res_data = static_cast &>(res_data_col).getData(); - - size_t size = src_offsets.size(); - res_offsets.resize(size); - res_data.reserve(src_data.size()); - - if (nullable) - res_null_map->reserve(src_null_map->size()); - - ColumnArray::Offset src_prev_offset = 0; - ColumnArray::Offset res_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - if (src_offsets[i] != src_prev_offset) - { - size_t size_to_write = src_offsets[i] - src_prev_offset; - res_data.resize(res_prev_offset + size_to_write); - memcpy(&res_data[res_prev_offset], &src_data[src_prev_offset], size_to_write * sizeof(T)); - - if (nullable) - { - res_null_map->resize(res_prev_offset + size_to_write); - memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); - } - - res_prev_offset += size_to_write; - res_offsets[i] = res_prev_offset; - } - else - { - res_data.push_back(T()); - ++res_prev_offset; - res_offsets[i] = res_prev_offset; - - if (nullable) - res_null_map->push_back(1); /// Push NULL. - } - - src_prev_offset = src_offsets[i]; - } - - return true; - } - else - return false; - } - - - template - bool executeFixedString( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, ColumnArray::Offsets & res_offsets, - const NullMap * src_null_map, - NullMap * res_null_map) - { - if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) - { - const size_t n = src_data_concrete->getN(); - const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); - - auto concrete_res_data = typeid_cast(&res_data_col); - if (!concrete_res_data) - throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; - - ColumnFixedString::Chars_t & res_data = concrete_res_data->getChars(); - size_t size = src_offsets.size(); - res_offsets.resize(size); - res_data.reserve(src_data.size()); - - if (nullable) - res_null_map->reserve(src_null_map->size()); - - ColumnArray::Offset src_prev_offset = 0; - ColumnArray::Offset res_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - if (src_offsets[i] != src_prev_offset) - { - size_t size_to_write = src_offsets[i] - src_prev_offset; - size_t prev_res_data_size = res_data.size(); - res_data.resize(prev_res_data_size + size_to_write * n); - memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset * n], size_to_write * n); - - if (nullable) - { - res_null_map->resize(res_prev_offset + size_to_write); - memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); - } - - res_prev_offset += size_to_write; - res_offsets[i] = res_prev_offset; - } - else - { - size_t prev_res_data_size = res_data.size(); - res_data.resize(prev_res_data_size + n); - memset(&res_data[prev_res_data_size], 0, n); - ++res_prev_offset; - res_offsets[i] = res_prev_offset; - - if (nullable) - res_null_map->push_back(1); - } - - src_prev_offset = src_offsets[i]; - } - - return true; - } - else - return false; - } - - - template - bool executeString( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets, - const NullMap * src_null_map, - NullMap * res_null_map) - { - if (const ColumnString * src_data_concrete = checkAndGetColumn(&src_data)) - { - const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); - - auto concrete_res_string_offsets = typeid_cast(&res_data_col); - if (!concrete_res_string_offsets) - throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; - ColumnString::Offsets & res_string_offsets = concrete_res_string_offsets->getOffsets(); - - const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); - - auto concrete_res_data = typeid_cast(&res_data_col); - if (!concrete_res_data) - throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; - ColumnString::Chars_t & res_data = concrete_res_data->getChars(); - - size_t size = src_array_offsets.size(); - res_array_offsets.resize(size); - res_string_offsets.reserve(src_string_offsets.size()); - res_data.reserve(src_data.size()); - - if (nullable) - res_null_map->reserve(src_null_map->size()); - - ColumnArray::Offset src_array_prev_offset = 0; - ColumnArray::Offset res_array_prev_offset = 0; - - ColumnString::Offset src_string_prev_offset = 0; - ColumnString::Offset res_string_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - if (src_array_offsets[i] != src_array_prev_offset) - { - size_t array_size = src_array_offsets[i] - src_array_prev_offset; - - size_t bytes_to_copy = 0; - size_t from_string_prev_offset_local = src_string_prev_offset; - for (size_t j = 0; j < array_size; ++j) - { - size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local; - - res_string_prev_offset += string_size; - res_string_offsets.push_back(res_string_prev_offset); - - from_string_prev_offset_local += string_size; - bytes_to_copy += string_size; - } - - size_t res_data_old_size = res_data.size(); - res_data.resize(res_data_old_size + bytes_to_copy); - memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy); - - if (nullable) - { - res_null_map->resize(res_array_prev_offset + array_size); - memcpy(&(*res_null_map)[res_array_prev_offset], &(*src_null_map)[src_array_prev_offset], array_size); - } - - res_array_prev_offset += array_size; - res_array_offsets[i] = res_array_prev_offset; - } - else - { - res_data.push_back(0); /// An empty string, including zero at the end. - - if (nullable) - res_null_map->push_back(1); - - ++res_string_prev_offset; - res_string_offsets.push_back(res_string_prev_offset); - - ++res_array_prev_offset; - res_array_offsets[i] = res_array_prev_offset; - } - - src_array_prev_offset = src_array_offsets[i]; - - if (src_array_prev_offset) - src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1]; - } - - return true; - } - else - return false; - } - - - template - void executeGeneric( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data, ColumnArray::Offsets & res_offsets, - const NullMap * src_null_map, - NullMap * res_null_map) - { - size_t size = src_offsets.size(); - res_offsets.resize(size); - res_data.reserve(src_data.size()); - - if (nullable) - res_null_map->reserve(src_null_map->size()); - - ColumnArray::Offset src_prev_offset = 0; - ColumnArray::Offset res_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - if (src_offsets[i] != src_prev_offset) - { - size_t size_to_write = src_offsets[i] - src_prev_offset; - res_data.insertRangeFrom(src_data, src_prev_offset, size_to_write); - - if (nullable) - { - res_null_map->resize(res_prev_offset + size_to_write); - memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); - } - - res_prev_offset += size_to_write; - res_offsets[i] = res_prev_offset; - } - else - { - res_data.insertDefault(); - ++res_prev_offset; - res_offsets[i] = res_prev_offset; - - if (nullable) - res_null_map->push_back(1); - } - - src_prev_offset = src_offsets[i]; - } - } - - - template - void executeDispatch( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets, - const NullMap * src_null_map, - NullMap * res_null_map) - { - if (!( executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeNumber(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeString (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) - || executeFixedString (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map))) - executeGeneric (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map); - } - } -} - -void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count)) - return; - - const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - if (!array) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - MutableColumnPtr res_ptr = array->cloneEmpty(); - ColumnArray & res = static_cast(*res_ptr); - - const IColumn & src_data = array->getData(); - const ColumnArray::Offsets & src_offsets = array->getOffsets(); - IColumn & res_data = res.getData(); - ColumnArray::Offsets & res_offsets = res.getOffsets(); - - const NullMap * src_null_map = nullptr; - NullMap * res_null_map = nullptr; - - const IColumn * inner_col; - IColumn * inner_res_col; - - bool nullable = src_data.isColumnNullable(); - if (nullable) - { - auto nullable_col = static_cast(&src_data); - inner_col = &nullable_col->getNestedColumn(); - src_null_map = &nullable_col->getNullMapData(); - - auto nullable_res_col = static_cast(&res_data); - inner_res_col = &nullable_res_col->getNestedColumn(); - res_null_map = &nullable_res_col->getNullMapData(); - } - else - { - inner_col = &src_data; - inner_res_col = &res_data; - } - - if (nullable) - FunctionEmptyArrayToSingleImpl::executeDispatch(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map); - else - FunctionEmptyArrayToSingleImpl::executeDispatch(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map); - - block.getByPosition(result).column = std::move(res_ptr); -} - - -/// Implementation of FunctionRange. - -String FunctionRange::getName() const -{ - return name; -} - -DataTypePtr FunctionRange::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypePtr & arg = arguments.front(); - - if (!arg->isUnsignedInteger()) - throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(arg); -} - -template -bool FunctionRange::executeInternal(Block & block, const IColumn * arg, const size_t result) -{ - static constexpr size_t max_elements = 100'000'000; - - if (const auto in = checkAndGetColumn>(arg)) - { - const auto & in_data = in->getData(); - const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{}, - [this] (const size_t lhs, const size_t rhs) - { - const auto sum = lhs + rhs; - if (sum < lhs) - throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", - ErrorCodes::ARGUMENT_OUT_OF_BOUND}; - - return sum; - }); - - if (total_values > max_elements) - throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + - " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), - ErrorCodes::ARGUMENT_OUT_OF_BOUND}; - - auto data_col = ColumnVector::create(total_values); - auto offsets_col = ColumnArray::ColumnOffsets::create(in->size()); - - auto & out_data = data_col->getData(); - auto & out_offsets = offsets_col->getData(); - - IColumn::Offset offset{}; - for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx) - { - for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx) - out_data[offset + elem_idx] = elem_idx; - - offset += in_data[row_idx]; - out_offsets[row_idx] = offset; - } - - block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); - return true; - } - else - return false; -} - -void FunctionRange::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto col = block.getByPosition(arguments[0]).column.get(); - - if (!executeInternal(block, col, result) && - !executeInternal(block, col, result) && - !executeInternal(block, col, result) && - !executeInternal(block, col, result)) - { - throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } -} - -/// Implementation of FunctionArrayReverse. - -FunctionPtr FunctionArrayReverse::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionArrayReverse::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayReverse::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception("Argument for function " + getName() + " must be array.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; -} - -void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - if (executeConst(block, arguments, result, input_rows_count)) - return; - - const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); - if (!array) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - auto res_ptr = array->cloneEmpty(); - ColumnArray & res = static_cast(*res_ptr); - res.getOffsetsPtr() = array->getOffsetsPtr(); - - const IColumn & src_data = array->getData(); - const ColumnArray::Offsets & offsets = array->getOffsets(); - IColumn & res_data = res.getData(); - - const ColumnNullable * nullable_col = nullptr; - ColumnNullable * nullable_res_col = nullptr; - - const IColumn * inner_col; - IColumn * inner_res_col; - - if (src_data.isColumnNullable()) - { - nullable_col = static_cast(&src_data); - inner_col = &nullable_col->getNestedColumn(); - - nullable_res_col = static_cast(&res_data); - inner_res_col = &nullable_res_col->getNestedColumn(); - } - else - { - inner_col = &src_data; - inner_res_col = &res_data; - } - - if (!( executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeString (*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeFixedString (*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col))) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - block.getByPosition(result).column = std::move(res_ptr); -} - -bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - size_t input_rows_count) -{ - if (const ColumnConst * const_array = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) - { - Array arr = const_array->getValue(); - - size_t size = arr.size(); - Array res(size); - - for (size_t i = 0; i < size; ++i) - res[i] = arr[size - i - 1]; - - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, res); - - return true; - } - else - return false; -} - -template -bool FunctionArrayReverse::executeNumber( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) -{ - auto do_reverse = [](const auto & src_data, const auto & src_offsets, auto & res_data) - { - size_t size = src_offsets.size(); - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const auto * src = &src_data[src_prev_offset]; - const auto * src_end = &src_data[src_offsets[i]]; - - if (src == src_end) - continue; - - auto dst = &res_data[src_offsets[i] - 1]; - - while (src < src_end) - { - *dst = *src; - ++src; - --dst; - } - - src_prev_offset = src_offsets[i]; - } - }; - - if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) - { - const PaddedPODArray & src_data = src_data_concrete->getData(); - PaddedPODArray & res_data = typeid_cast &>(res_data_col).getData(); - res_data.resize(src_data.size()); - do_reverse(src_data, src_offsets, res_data); - - if ((nullable_col) && (nullable_res_col)) - { - /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); - res_null_map.resize(src_data.size()); - do_reverse(src_null_map, src_offsets, res_null_map); - } - - return true; - } - else - return false; -} - -bool FunctionArrayReverse::executeFixedString( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) -{ - if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) - { - const size_t n = src_data_concrete->getN(); - const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnFixedString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); - size_t size = src_offsets.size(); - res_data.resize(src_data.size()); - - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const UInt8 * src = &src_data[src_prev_offset * n]; - const UInt8 * src_end = &src_data[src_offsets[i] * n]; - - if (src == src_end) - continue; - - UInt8 * dst = &res_data[src_offsets[i] * n - n]; - - while (src < src_end) - { - /// NOTE: memcpySmallAllowReadWriteOverflow15 doesn't work correctly here. - memcpy(dst, src, n); - src += n; - dst -= n; - } - - src_prev_offset = src_offsets[i]; - } - - if ((nullable_col) && (nullable_res_col)) - { - /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); - res_null_map.resize(src_null_map.size()); - - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const UInt8 * src = &src_null_map[src_prev_offset]; - const UInt8 * src_end = &src_null_map[src_offsets[i]]; - - if (src == src_end) - continue; - - UInt8 * dst = &res_null_map[src_offsets[i] - 1]; - - while (src < src_end) - { - *dst = *src; - ++src; - --dst; - } - - src_prev_offset = src_offsets[i]; - } - } - - return true; - } - else - return false; -} - -bool FunctionArrayReverse::executeString( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) -{ - if (const ColumnString * src_data_concrete = checkAndGetColumn(&src_data)) - { - const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); - ColumnString::Offsets & res_string_offsets = typeid_cast(res_data_col).getOffsets(); - - const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); - - size_t size = src_array_offsets.size(); - res_string_offsets.resize(src_string_offsets.size()); - res_data.resize(src_data.size()); - - ColumnArray::Offset src_array_prev_offset = 0; - ColumnString::Offset res_string_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - if (src_array_offsets[i] != src_array_prev_offset) - { - size_t array_size = src_array_offsets[i] - src_array_prev_offset; - - for (size_t j = 0; j < array_size; ++j) - { - size_t j_reversed = array_size - j - 1; - - auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1]; - size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos; - - memcpySmallAllowReadWriteOverflow15(&res_data[res_string_prev_offset], &src_data[src_pos], string_size); - - res_string_prev_offset += string_size; - res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset; - } - } - - src_array_prev_offset = src_array_offsets[i]; - } - - if ((nullable_col) && (nullable_res_col)) - { - /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); - res_null_map.resize(src_string_offsets.size()); - - size_t size = src_string_offsets.size(); - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const auto * src = &src_null_map[src_prev_offset]; - const auto * src_end = &src_null_map[src_array_offsets[i]]; - - if (src == src_end) - continue; - - auto dst = &res_null_map[src_array_offsets[i] - 1]; - - while (src < src_end) - { - *dst = *src; - ++src; - --dst; - } - - src_prev_offset = src_array_offsets[i]; - } - } - - return true; - } - else - return false; -} - -/// Implementation of FunctionArrayReduce. - -FunctionPtr FunctionArrayReduce::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionArrayReduce::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const -{ - /// The first argument is a constant string with the name of the aggregate function - /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). - - if (arguments.size() < 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be at least 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst(arguments[0].column.get()); - if (!aggregate_function_name_column) - throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - DataTypes argument_types(arguments.size() - 1); - for (size_t i = 1, size = arguments.size(); i < size; ++i) - { - const DataTypeArray * arg = checkAndGetDataType(arguments[i].type.get()); - if (!arg) - throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type " - + arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - argument_types[i - 1] = arg->getNestedType(); - } - - if (!aggregate_function) - { - String aggregate_function_name_with_params = aggregate_function_name_column->getValue(); - - if (aggregate_function_name_with_params.empty()) - throw Exception("First argument for function " + getName() + " (name of aggregate function) cannot be empty.", - ErrorCodes::BAD_ARGUMENTS); - - String aggregate_function_name; - Array params_row; - getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params, - aggregate_function_name, params_row, "function " + getName()); - - aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row); - } - - return aggregate_function->getReturnType(); -} - - - -void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - IAggregateFunction & agg_func = *aggregate_function.get(); - AlignedBuffer place_holder(agg_func.sizeOfData(), agg_func.alignOfData()); - AggregateDataPtr place = place_holder.data(); - - std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; - - size_t rows = input_rows_count; - - /// Aggregate functions do not support constant columns. Therefore, we materialize them. - std::vector materialized_columns; - - const size_t num_arguments_columns = arguments.size() - 1; - - std::vector aggregate_arguments_vec(num_arguments_columns); - const ColumnArray::Offsets * offsets = nullptr; - - for (size_t i = 0; i < num_arguments_columns; ++i) - { - const IColumn * col = block.getByPosition(arguments[i + 1]).column.get(); - const ColumnArray::Offsets * offsets_i = nullptr; - if (const ColumnArray * arr = checkAndGetColumn(col)) - { - aggregate_arguments_vec[i] = &arr->getData(); - offsets_i = &arr->getOffsets(); - } - else if (const ColumnConst * const_arr = checkAndGetColumnConst(col)) - { - materialized_columns.emplace_back(const_arr->convertToFullColumn()); - const auto & arr = typeid_cast(*materialized_columns.back().get()); - aggregate_arguments_vec[i] = &arr.getData(); - offsets_i = &arr.getOffsets(); - } - else - throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - - if (i == 0) - offsets = offsets_i; - else if (*offsets_i != *offsets) - throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", - ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - } - const IColumn ** aggregate_arguments = aggregate_arguments_vec.data(); - - MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn(); - IColumn & res_col = *result_holder; - - /// AggregateFunction's states should be inserted into column using specific way - auto res_col_aggregate_function = typeid_cast(&res_col); - - if (!res_col_aggregate_function && agg_func.isState()) - throw Exception("State function " + agg_func.getName() + " inserts results into non-state column " - + block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < rows; ++i) - { - agg_func.create(place); - ColumnArray::Offset next_offset = (*offsets)[i]; - - try - { - for (size_t j = current_offset; j < next_offset; ++j) - agg_func.add(place, aggregate_arguments, j, arena.get()); - - if (!res_col_aggregate_function) - agg_func.insertResultInto(place, res_col); - else - res_col_aggregate_function->insertFrom(place); - } - catch (...) - { - agg_func.destroy(place); - throw; - } - - agg_func.destroy(place); - current_offset = next_offset; - } - - block.getByPosition(result).column = std::move(result_holder); -} - -/// Implementation of FunctionArrayConcat. - -FunctionPtr FunctionArrayConcat::create(const Context & context) -{ - return std::make_shared(context); -} - -String FunctionArrayConcat::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayConcat::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments.empty()) - throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - for (auto i : ext::range(0, arguments.size())) - { - auto array_type = typeid_cast(arguments[i].get()); - if (!array_type) - throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " - + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return getLeastSupertype(arguments); -} - -void FunctionArrayConcat::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const DataTypePtr & return_type = block.getByPosition(result).type; - - if (return_type->onlyNull()) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto result_column = return_type->createColumn(); - - size_t rows = input_rows_count; - size_t num_args = arguments.size(); - - Columns preprocessed_columns(num_args); - - for (size_t i = 0; i < num_args; ++i) - { - const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]); - ColumnPtr preprocessed_column = arg.column; - - if (!arg.type->equals(*return_type)) - preprocessed_column = castColumn(arg, return_type, context); - - preprocessed_columns[i] = std::move(preprocessed_column); - } - - std::vector> sources; - - for (auto & argument_column : preprocessed_columns) - { - bool is_const = false; - - if (auto argument_column_const = typeid_cast(argument_column.get())) - { - is_const = true; - argument_column = argument_column_const->getDataColumnPtr(); - } - - if (auto argument_column_array = typeid_cast(argument_column.get())) - sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows)); - else - throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; - } - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), rows); - GatherUtils::concat(sources, *sink); - - block.getByPosition(result).column = std::move(result_column); -} - - -/// Implementation of FunctionArraySlice. - -FunctionPtr FunctionArraySlice::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionArraySlice::getName() const -{ - return name; -} - -DataTypePtr FunctionArraySlice::getReturnTypeImpl(const DataTypes & arguments) const -{ - const size_t number_of_arguments = arguments.size(); - - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(number_of_arguments) + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (arguments[0]->onlyNull()) - return arguments[0]; - - auto array_type = typeid_cast(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be an array but it has type " - + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - for (size_t i = 1; i < number_of_arguments; ++i) - { - if (!removeNullable(arguments[i])->isInteger() && !arguments[i]->onlyNull()) - throw Exception( - "Argument " + toString(i) + " for function " + getName() + " must be integer but it has type " - + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return arguments[0]; -} - -void FunctionArraySlice::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const auto & return_type = block.getByPosition(result).type; - - if (return_type->onlyNull()) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto result_column = return_type->createColumn(); - - auto & array_column = block.getByPosition(arguments[0]).column; - const auto & offset_column = block.getByPosition(arguments[1]).column; - const auto & length_column = arguments.size() > 2 ? block.getByPosition(arguments[2]).column : nullptr; - - std::unique_ptr source; - - size_t size = array_column->size(); - bool is_const = false; - - if (auto const_array_column = typeid_cast(array_column.get())) - { - is_const = true; - array_column = const_array_column->getDataColumnPtr(); - } - - if (auto argument_column_array = typeid_cast(array_column.get())) - source = GatherUtils::createArraySource(*argument_column_array, is_const, size); - else - throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); - - if (offset_column->onlyNull()) - { - if (!length_column || length_column->onlyNull()) - { - block.getByPosition(result).column = array_column; - return; - } - else if (length_column->isColumnConst()) - GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, length_column->getInt(0)); - else - { - auto const_offset_column = ColumnConst::create(ColumnInt8::create(1, 1), size); - GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *const_offset_column, *length_column); - } - } - else if (offset_column->isColumnConst()) - { - ssize_t offset = offset_column->getUInt(0); - - if (!length_column || length_column->onlyNull()) - { - if (offset > 0) - GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, static_cast(offset - 1)); - else - GatherUtils::sliceFromRightConstantOffsetUnbounded(*source, *sink, static_cast(-offset)); - } - else if (length_column->isColumnConst()) - { - ssize_t length = length_column->getInt(0); - if (offset > 0) - GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, static_cast(offset - 1), length); - else - GatherUtils::sliceFromRightConstantOffsetBounded(*source, *sink, static_cast(-offset), length); - } - else - GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column); - } - else - { - if (!length_column || length_column->onlyNull()) - GatherUtils::sliceDynamicOffsetUnbounded(*source, *sink, *offset_column); - else - GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column); - } - - block.getByPosition(result).column = std::move(result_column); -} - - -/// Implementation of FunctionArrayPush. - -DataTypePtr FunctionArrayPush::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments[0]->onlyNull()) - return arguments[0]; - - auto array_type = typeid_cast(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be an array but it has type " - + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto nested_type = array_type->getNestedType(); - - DataTypes types = {nested_type, arguments[1]}; - - return std::make_shared(getLeastSupertype(types)); -} - -void FunctionArrayPush::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const auto & return_type = block.getByPosition(result).type; - - if (return_type->onlyNull()) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto result_column = return_type->createColumn(); - - auto array_column = block.getByPosition(arguments[0]).column; - auto appended_column = block.getByPosition(arguments[1]).column; - - if (!block.getByPosition(arguments[0]).type->equals(*return_type)) - array_column = castColumn(block.getByPosition(arguments[0]), return_type, context); - - const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); - if (!block.getByPosition(arguments[1]).type->equals(*return_nested_type)) - appended_column = castColumn(block.getByPosition(arguments[1]), return_nested_type, context); - - std::unique_ptr array_source; - std::unique_ptr value_source; - - size_t size = array_column->size(); - bool is_const = false; - - if (auto const_array_column = typeid_cast(array_column.get())) - { - is_const = true; - array_column = const_array_column->getDataColumnPtr(); - } - - if (auto argument_column_array = typeid_cast(array_column.get())) - array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); - else - throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; - - - bool is_appended_const = false; - if (auto const_appended_column = typeid_cast(appended_column.get())) - { - is_appended_const = true; - appended_column = const_appended_column->getDataColumnPtr(); - } - - value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); - - GatherUtils::push(*array_source, *value_source, *sink, push_front); - - block.getByPosition(result).column = std::move(result_column); -} - -/// Implementation of FunctionArrayPushFront. - -FunctionPtr FunctionArrayPushFront::create(const Context & context) -{ - return std::make_shared(context); -} - -/// Implementation of FunctionArrayPushBack. - -FunctionPtr FunctionArrayPushBack::create(const Context & context) -{ - return std::make_shared(context); -} - -/// Implementation of FunctionArrayPop. - -DataTypePtr FunctionArrayPop::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments[0]->onlyNull()) - return arguments[0]; - - auto array_type = typeid_cast(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be an array but it has type " - + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; -} - -void FunctionArrayPop::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const auto & return_type = block.getByPosition(result).type; - - if (return_type->onlyNull()) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto result_column = return_type->createColumn(); - - const auto & array_column = block.getByPosition(arguments[0]).column; - - std::unique_ptr source; - - size_t size = array_column->size(); - - if (auto argument_column_array = typeid_cast(array_column.get())) - source = GatherUtils::createArraySource(*argument_column_array, false, size); - else - throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); - - if (pop_front) - GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, 1); - else - GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, -1); - - block.getByPosition(result).column = std::move(result_column); -} - -/// Implementation of FunctionArrayPopFront. - -FunctionPtr FunctionArrayPopFront::create(const Context &) -{ - return std::make_shared(); -} - -/// Implementation of FunctionArrayPopBack. - -FunctionPtr FunctionArrayPopBack::create(const Context &) -{ - return std::make_shared(); -} - - -/// Implementation of FunctionArrayAllAny. - -FunctionPtr FunctionArrayHasAll::create(const Context & context) -{ - return std::make_shared(context); -} - -FunctionPtr FunctionArrayHasAny::create(const Context & context) -{ - return std::make_shared(context); -} - - -DataTypePtr FunctionArrayHasAllAny::getReturnTypeImpl(const DataTypes & arguments) const -{ - for (auto i : ext::range(0, arguments.size())) - { - auto array_type = typeid_cast(arguments[i].get()); - if (!array_type) - throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " - + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return std::make_shared(); -} - -void FunctionArrayHasAllAny::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - size_t rows = input_rows_count; - size_t num_args = arguments.size(); - - auto result_column = ColumnUInt8::create(rows); - - DataTypePtr common_type = nullptr; - auto commonType = [& common_type, & block, & arguments]() - { - if (common_type == nullptr) - { - DataTypes data_types; - data_types.reserve(arguments.size()); - for (const auto & argument : arguments) - data_types.push_back(block.getByPosition(argument).type); - - common_type = getLeastSupertype(data_types); - } - - return common_type; - }; - - Columns preprocessed_columns(num_args); - - for (size_t i = 0; i < num_args; ++i) - { - const auto & argument = block.getByPosition(arguments[i]); - ColumnPtr preprocessed_column = argument.column; - - const auto argument_type = typeid_cast(argument.type.get()); - const auto & nested_type = argument_type->getNestedType(); - - /// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1 - if (typeid_cast(removeNullable(nested_type).get())) - preprocessed_column = castColumn(argument, commonType(), context); - - preprocessed_columns[i] = std::move(preprocessed_column); - } - - std::vector> sources; - - for (auto & argument_column : preprocessed_columns) - { - bool is_const = false; - - if (auto argument_column_const = typeid_cast(argument_column.get())) - { - is_const = true; - argument_column = argument_column_const->getDataColumnPtr(); - } - - if (auto argument_column_array = typeid_cast(argument_column.get())) - sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows)); - else - throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; - } - - auto result_column_ptr = typeid_cast(result_column.get()); - GatherUtils::sliceHas(*sources[0], *sources[1], all, *result_column_ptr); - - block.getByPosition(result).column = std::move(result_column); -} - -/// Implementation of FunctionArrayIntersect. - -FunctionPtr FunctionArrayIntersect::create(const Context & context) -{ - return std::make_shared(context); -} - -String FunctionArrayIntersect::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayIntersect::getReturnTypeImpl(const DataTypes & arguments) const -{ - DataTypes nested_types; - nested_types.reserve(arguments.size()); - - bool has_nothing = false; - - if (arguments.empty()) - throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - for (auto i : ext::range(0, arguments.size())) - { - auto array_type = typeid_cast(arguments[i].get()); - if (!array_type) - throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " - + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto & nested_type = array_type->getNestedType(); - - if (typeid_cast(nested_type.get())) - has_nothing = true; - else - nested_types.push_back(nested_type); - } - - DataTypePtr result_type; - - if (!nested_types.empty()) - result_type = getMostSubtype(nested_types, true); - - if (has_nothing) - result_type = std::make_shared(); - - return std::make_shared(result_type); -} - -ColumnPtr FunctionArrayIntersect::castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const -{ - if (auto column_nullable = checkAndGetColumn(column.get())) - { - auto nullable_type = checkAndGetDataType(data_type.get()); - const auto & nested = column_nullable->getNestedColumnPtr(); - if (nullable_type) - { - auto casted_column = castRemoveNullable(nested, nullable_type->getNestedType()); - return ColumnNullable::create(casted_column, column_nullable->getNullMapColumnPtr()); - } - return castRemoveNullable(nested, data_type); - } - else if (auto column_array = checkAndGetColumn(column.get())) - { - auto array_type = checkAndGetDataType(data_type.get()); - if (!array_type) - throw Exception{"Cannot cast array column to column with type " - + data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR}; - - auto casted_column = castRemoveNullable(column_array->getDataPtr(), array_type->getNestedType()); - return ColumnArray::create(casted_column, column_array->getOffsetsPtr()); - } - else if (auto column_tuple = checkAndGetColumn(column.get())) - { - auto tuple_type = checkAndGetDataType(data_type.get()); - - if (!tuple_type) - throw Exception{"Cannot cast tuple column to type " - + data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR}; - - auto columns_number = column_tuple->getColumns().size(); - Columns columns(columns_number); - - const auto & types = tuple_type->getElements(); - - for (auto i : ext::range(0, columns_number)) - { - columns[i] = castRemoveNullable(column_tuple->getColumnPtr(i), types[i]); - } - return ColumnTuple::create(columns); - } - - return column; -} - -Columns FunctionArrayIntersect::castColumns( - Block & block, const ColumnNumbers & arguments, const DataTypePtr & return_type, - const DataTypePtr & return_type_with_nulls) const -{ - size_t num_args = arguments.size(); - Columns columns(num_args); - - auto type_array = checkAndGetDataType(return_type.get()); - auto & type_nested = type_array->getNestedType(); - auto type_not_nullable_nested = removeNullable(type_nested); - - const bool is_numeric_or_string = type_not_nullable_nested->isNumber() - || type_not_nullable_nested->isDateOrDateTime() - || type_not_nullable_nested->isStringOrFixedString(); - - DataTypePtr nullable_return_type; - - if (is_numeric_or_string) - { - auto type_nullable_nested = makeNullable(type_nested); - nullable_return_type = std::make_shared(type_nullable_nested); - } - - const bool nested_is_nullable = type_nested->isNullable(); - - for (size_t i = 0; i < num_args; ++i) - { - const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]); - auto & column = columns[i]; - - if (is_numeric_or_string) - { - /// Cast to Array(T) or Array(Nullable(T)). - if (nested_is_nullable) - { - if (arg.type->equals(*return_type)) - column = arg.column; - else - column = castColumn(arg, return_type, context); - } - else - { - /// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T)) - /// because cannot cast Nullable(T) to T. - if (arg.type->equals(*return_type) || arg.type->equals(*nullable_return_type)) - column = arg.column; - else if (static_cast(*arg.type).getNestedType()->isNullable()) - column = castColumn(arg, nullable_return_type, context); - else - column = castColumn(arg, return_type, context); - } - } - else - { - /// return_type_with_nulls is the most common subtype with possible nullable parts. - if (arg.type->equals(*return_type_with_nulls)) - column = arg.column; - else - column = castColumn(arg, return_type_with_nulls, context); - } - } - - return columns; -} - -FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(const Columns & columns) const -{ - UnpackedArrays arrays; - - size_t columns_number = columns.size(); - arrays.is_const.assign(columns_number, false); - arrays.null_maps.resize(columns_number); - arrays.offsets.resize(columns_number); - arrays.nested_columns.resize(columns_number); - - for (auto i : ext::range(0, columns_number)) - { - auto argument_column = columns[i].get(); - if (auto argument_column_const = typeid_cast(argument_column)) - { - arrays.is_const[i] = true; - argument_column = argument_column_const->getDataColumnPtr().get(); - } - - if (auto argument_column_array = typeid_cast(argument_column)) - { - arrays.offsets[i] = &argument_column_array->getOffsets(); - arrays.nested_columns[i] = &argument_column_array->getData(); - if (auto column_nullable = typeid_cast(arrays.nested_columns[i])) - { - arrays.null_maps[i] = &column_nullable->getNullMapData(); - arrays.nested_columns[i] = &column_nullable->getNestedColumn(); - } - } - else - throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; - } - - return arrays; -} - -void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const auto & return_type = block.getByPosition(result).type; - auto return_type_array = checkAndGetDataType(return_type.get()); - - if (!return_type) - throw Exception{"Return type for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; - - const auto & nested_return_type = return_type_array->getNestedType(); - - if (typeid_cast(nested_return_type.get())) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto num_args = arguments.size(); - DataTypes data_types; - data_types.reserve(num_args); - for (size_t i = 0; i < num_args; ++i) - data_types.push_back(block.getByPosition(arguments[i]).type); - - auto return_type_with_nulls = getMostSubtype(data_types, true, true); - - Columns columns = castColumns(block, arguments, return_type, return_type_with_nulls); - - UnpackedArrays arrays = prepareArrays(columns); - - ColumnPtr result_column; - auto not_nullable_nested_return_type = removeNullable(nested_return_type); - TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column)); - - using DateMap = ClearableHashMap, - HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDate::FieldType)>>; - - using DateTimeMap = ClearableHashMap, - HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDateTime::FieldType)>>; - - using StringMap = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - - if (!result_column) - { - auto column = not_nullable_nested_return_type->createColumn(); - - if (checkDataType(not_nullable_nested_return_type.get())) - result_column = execute, true>(arrays, std::move(column)); - else if (checkDataType(not_nullable_nested_return_type.get())) - result_column = execute, true>(arrays, std::move(column)); - else if(not_nullable_nested_return_type->isString()) - result_column = execute(arrays, std::move(column)); - else if(not_nullable_nested_return_type->isFixedString()) - result_column = execute(arrays, std::move(column)); - else - { - column = static_cast(*return_type_with_nulls).getNestedType()->createColumn(); - result_column = castRemoveNullable(execute(arrays, std::move(column)), return_type); - } - } - - block.getByPosition(result).column = std::move(result_column); -} - -template -void FunctionArrayIntersect::NumberExecutor::operator()() -{ - using Map = ClearableHashMap, HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - - if (!result && typeid_cast *>(data_type.get())) - result = execute, true>(arrays, ColumnVector::create()); -} - -template -ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr) -{ - auto args = arrays.nested_columns.size(); - auto rows = arrays.offsets.front()->size(); - - bool all_nullable = true; - - std::vector columns; - columns.reserve(args); - for (auto arg : ext::range(0, args)) - { - if constexpr (std::is_same::value) - columns.push_back(arrays.nested_columns[arg]); - else - columns.push_back(checkAndGetColumn(arrays.nested_columns[arg])); - - if (!columns.back()) - throw Exception("Unexpected array type for function arrayIntersect", ErrorCodes::LOGICAL_ERROR); - - if (!arrays.null_maps[arg]) - all_nullable = false; - } - - auto & result_data = static_cast(*result_data_ptr); - auto result_offsets_ptr = ColumnArray::ColumnOffsets::create(rows); - auto & result_offsets = static_cast(*result_offsets_ptr); - auto null_map_column = ColumnUInt8::create(); - NullMap & null_map = static_cast(*null_map_column).getData(); - - Arena arena; - - Map map; - std::vector prev_off(args, 0); - size_t result_offset = 0; - for (auto row : ext::range(0, rows)) - { - map.clear(); - - bool all_has_nullable = all_nullable; - - for (auto arg : ext::range(0, args)) - { - bool current_has_nullable = false; - size_t off = (*arrays.offsets[arg])[row]; - for (auto i : ext::range(prev_off[arg], off)) - { - if (arrays.null_maps[arg] && (*arrays.null_maps[arg])[i]) - current_has_nullable = true; - else - { - if constexpr (is_numeric_column) - ++map[columns[arg]->getElement(i)]; - else if constexpr (std::is_same::value || std::is_same::value) - ++map[columns[arg]->getDataAt(i)]; - else - { - const char * data = nullptr; - ++map[columns[arg]->serializeValueIntoArena(i, arena, data)]; - } - } - } - - prev_off[arg] = off; - if (!current_has_nullable) - all_has_nullable = false; - } - - if (all_has_nullable) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); - } - - for (const auto & pair : map) - { - if (pair.second == args) - { - ++result_offset; - if constexpr (is_numeric_column) - result_data.insert(pair.first); - else if constexpr (std::is_same::value || std::is_same::value) - result_data.insertData(pair.first.data, pair.first.size); - else - result_data.deserializeAndInsertFromArena(pair.first.data); - - if (all_nullable) - null_map.push_back(0); - } - } - result_offsets.getElement(row) = result_offset; - } - - ColumnPtr result_column = std::move(result_data_ptr); - if (all_nullable) - result_column = ColumnNullable::create(result_column, std::move(null_map_column)); - return ColumnArray::create(result_column, std::move(result_offsets_ptr)); -} - -/// Implementation of FunctionArrayResize. - -FunctionPtr FunctionArrayResize::create(const Context & context) -{ - return std::make_shared(context); -} - -String FunctionArrayResize::getName() const -{ - return name; -} - -DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments) const -{ - const size_t number_of_arguments = arguments.size(); - - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(number_of_arguments) + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (arguments[0]->onlyNull()) - return arguments[0]; - - auto array_type = typeid_cast(arguments[0].get()); - if (!array_type) - throw Exception("First argument for function " + getName() + " must be an array but it has type " - + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (checkDataType(array_type->getNestedType().get())) - throw Exception("Function " + getName() + " cannot resize " + array_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!removeNullable(arguments[1])->isInteger() && !arguments[1]->onlyNull()) - throw Exception( - "Argument " + toString(1) + " for function " + getName() + " must be integer but it has type " - + arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (number_of_arguments == 2) - return arguments[0]; - else /* if (number_of_arguments == 3) */ - return std::make_shared(getLeastSupertype({array_type->getNestedType(), arguments[2]})); -} - -void FunctionArrayResize::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - const auto & return_type = block.getByPosition(result).type; - - if (return_type->onlyNull()) - { - block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto result_column = return_type->createColumn(); - - auto array_column = block.getByPosition(arguments[0]).column; - auto size_column = block.getByPosition(arguments[1]).column; - - if (!block.getByPosition(arguments[0]).type->equals(*return_type)) - array_column = castColumn(block.getByPosition(arguments[0]), return_type, context); - - const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); - size_t size = array_column->size(); - - ColumnPtr appended_column; - if (arguments.size() == 3) - { - appended_column = block.getByPosition(arguments[2]).column; - if (!block.getByPosition(arguments[2]).type->equals(*return_nested_type)) - appended_column = castColumn(block.getByPosition(arguments[2]), return_nested_type, context); - } - else - appended_column = return_nested_type->createColumnConstWithDefaultValue(size); - - std::unique_ptr array_source; - std::unique_ptr value_source; - - bool is_const = false; - - if (auto const_array_column = typeid_cast(array_column.get())) - { - is_const = true; - array_column = const_array_column->getDataColumnPtr(); - } - - if (auto argument_column_array = typeid_cast(array_column.get())) - array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); - else - throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; - - - bool is_appended_const = false; - if (auto const_appended_column = typeid_cast(appended_column.get())) - { - is_appended_const = true; - appended_column = const_appended_column->getDataColumnPtr(); - } - - value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); - - if (size_column->isColumnConst()) - GatherUtils::resizeConstantSize(*array_source, *value_source, *sink, size_column->getInt(0)); - else - GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column); - - block.getByPosition(result).column = std::move(result_column); -} - - -} diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 1ed1ca59173..d3ae5d86f76 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -150,12 +150,12 @@ public: ", expected FixedString(" + toString(ipv6_bytes_length) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!checkDataType(arguments[1].get())) + if (!WhichDataType(arguments[1]).isUInt8()) throw Exception("Illegal type " + arguments[1]->getName() + " of argument 2 of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt8()) throw Exception("Illegal type " + arguments[2]->getName() + " of argument 3 of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -266,7 +266,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -519,7 +519,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!checkDataType(arguments[0].get())) + if (!WhichDataType(arguments[0]).isUInt32()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -579,7 +579,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -714,7 +714,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!checkDataType(arguments[0].get())) + if (!WhichDataType(arguments[0]).isUInt64()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt64", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -843,7 +843,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1006,7 +1006,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { /// String or FixedString(36) - if (!arguments[0]->isString()) + if (!isString(arguments[0])) { const auto ptr = checkAndGetDataType(arguments[0].get()); if (!ptr || ptr->getN() != uuid_text_length) @@ -1151,13 +1151,11 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString() - && !arguments[0]->isFixedString() - && !arguments[0]->isDateOrDateTime() - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get())) + WhichDataType which(arguments[0]); + + if (!which.isStringOrFixedString() + && !which.isDateOrDateTime() + && !which.isUInt()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1370,7 +1368,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1460,7 +1458,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isInteger()) + if (!isInteger(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1543,7 +1541,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index a51fca02b93..1d8a1e3296c 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -817,7 +817,7 @@ private: const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped; const IDataType * number_type = left_is_num ? left_type.get() : right_type.get(); - DataTypeExtractor which(number_type); + WhichDataType which(number_type); const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID(); @@ -1077,8 +1077,8 @@ public: /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - DataTypeExtractor left(arguments[0].get()); - DataTypeExtractor right(arguments[1].get()); + WhichDataType left(arguments[0].get()); + WhichDataType right(arguments[1].get()); const DataTypeTuple * left_tuple = checkAndGetDataType(arguments[0].get()); const DataTypeTuple * right_tuple = checkAndGetDataType(arguments[1].get()); @@ -1159,9 +1159,9 @@ public: { executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count); } - else if (isDecimal(left_type.get()) || isDecimal(right_type.get())) + else if (isDecimal(left_type) || isDecimal(right_type)) { - if (!allowDecimalComparison(left_type.get(), right_type.get())) + if (!allowDecimalComparison(left_type, right_type)) throw Exception("No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1193,7 +1193,7 @@ public: auto isFloatingPoint = &typeIsEither; if ((isBigInteger(*types[0]) && isFloatingPoint(*types[1])) || (isBigInteger(*types[1]) && isFloatingPoint(*types[0]))) return false; /// TODO: implement (double, int_N where N > double's mantissa width) - return isCompilableType(types[0].get()) && isCompilableType(types[1].get()); + return isCompilableType(types[0]) && isCompilableType(types[1]); } llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override diff --git a/dbms/src/Functions/FunctionsConditional.cpp b/dbms/src/Functions/FunctionsConditional.cpp index 4afdbc26acc..92ae0a5b0fb 100644 --- a/dbms/src/Functions/FunctionsConditional.cpp +++ b/dbms/src/Functions/FunctionsConditional.cpp @@ -1,8 +1,9 @@ #include -#include #include #include #include +#include +#include #include #include @@ -205,7 +206,7 @@ DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const nested_type = arg.get(); } - if (!checkDataType(nested_type)) + if (!WhichDataType(nested_type).isUInt8()) throw Exception{"Illegal type " + arg->getName() + " of argument (condition) " "of function " + getName() + ". Must be UInt8.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -251,22 +252,15 @@ DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args /// See the comments in executeImpl() to understand why we actually have to /// get the return type of a transform function. - /// Get the return types of the arrays that we pass to the transform function. - ColumnsWithTypeAndName src_array_types; - ColumnsWithTypeAndName dst_array_types; + /// Get the types of the arrays that we pass to the transform function. + DataTypes src_array_types; + DataTypes dst_array_types; - for (size_t i = 1; i < (args.size() - 1); ++i) - { - if ((i % 2) != 0) - src_array_types.push_back({nullptr, args[i], {}}); - else - dst_array_types.push_back({nullptr, args[i], {}}); - } + for (size_t i = 1; i < args.size() - 1; ++i) + ((i % 2) ? src_array_types : dst_array_types).push_back(args[i]); - FunctionArray fun_array{context}; - - DataTypePtr src_array_type = fun_array.getReturnType(src_array_types); - DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types); + DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); + DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); /// Finally get the return type of the transform function. FunctionTransform fun_transform; @@ -291,29 +285,31 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers /// Create the arrays required by the transform function. ColumnNumbers src_array_args; - ColumnsWithTypeAndName src_array_types; + ColumnsWithTypeAndName src_array_elems; + DataTypes src_array_types; ColumnNumbers dst_array_args; - ColumnsWithTypeAndName dst_array_types; + ColumnsWithTypeAndName dst_array_elems; + DataTypes dst_array_types; for (size_t i = 1; i < (args.size() - 1); ++i) { - if ((i % 2) != 0) + if (i % 2) { src_array_args.push_back(args[i]); - src_array_types.push_back(block.getByPosition(args[i])); + src_array_elems.push_back(block.getByPosition(args[i])); + src_array_types.push_back(block.getByPosition(args[i]).type); } else { dst_array_args.push_back(args[i]); - dst_array_types.push_back(block.getByPosition(args[i])); + dst_array_elems.push_back(block.getByPosition(args[i])); + dst_array_types.push_back(block.getByPosition(args[i]).type); } } - FunctionArray fun_array{context}; - - DataTypePtr src_array_type = fun_array.getReturnType(src_array_types); - DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types); + DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); + DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); Block temp_block = block; @@ -323,8 +319,10 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers size_t dst_array_pos = temp_block.columns(); temp_block.insert({nullptr, dst_array_type, ""}); - fun_array.execute(temp_block, src_array_args, src_array_pos, input_rows_count); - fun_array.execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); + auto fun_array = FunctionFactory::instance().get("array", context); + + fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count); + fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); /// Execute transform. FunctionTransform fun_transform; diff --git a/dbms/src/Functions/FunctionsConditional.h b/dbms/src/Functions/FunctionsConditional.h index e8d261216f4..865f98c935b 100644 --- a/dbms/src/Functions/FunctionsConditional.h +++ b/dbms/src/Functions/FunctionsConditional.h @@ -122,7 +122,7 @@ public: bool isCompilableImpl(const DataTypes & types) const override { for (const auto & type : types) - if (!isCompilableType(removeNullable(type).get())) + if (!isCompilableType(removeNullable(type))) return false; return true; } @@ -895,7 +895,7 @@ public: return makeNullable(getReturnTypeImpl({ removeNullable(arguments[0]), arguments[1], arguments[2]})); - if (!checkDataType(arguments[0].get())) + if (!WhichDataType(arguments[0]).isUInt8()) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument (condition) of function if. Must be UInt8.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsConsistentHashing.h b/dbms/src/Functions/FunctionsConsistentHashing.h index 5c5c2cbc86f..7961395638f 100644 --- a/dbms/src/Functions/FunctionsConsistentHashing.h +++ b/dbms/src/Functions/FunctionsConsistentHashing.h @@ -106,7 +106,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isInteger()) + if (!isInteger(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -115,7 +115,7 @@ public: + ", got " + arguments[0]->getName(), ErrorCodes::BAD_ARGUMENTS); - if (!arguments[1]->isInteger()) + if (!isInteger(arguments[1])) throw Exception("Illegal type " + arguments[1]->getName() + " of the second argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -178,21 +178,23 @@ private: const IDataType * hash_type = block.getByPosition(arguments[0]).type.get(); auto res_col = ColumnVector::create(); - if (checkDataType(hash_type)) + WhichDataType which(hash_type); + + if (which.isUInt8()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isUInt16()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isUInt32()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isUInt64()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isInt8()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isInt16()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isInt32()) executeType(hash_col, num_buckets, res_col.get()); - else if (checkDataType(hash_type)) + else if (which.isInt64()) executeType(hash_col, num_buckets, res_col.get()); else throw Exception("Illegal type " + hash_type->getName() + " of the first argument of function " + getName(), diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index 930212922de..fdfc153f594 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -20,7 +20,7 @@ void throwExceptionForIncompletelyParsedValue( else message_buf << " at begin of string"; - if (to_type.isNumber()) + if (isNumber(to_type)) message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index fd0159b2b66..a8dd3b6d033 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -802,7 +802,7 @@ public: || std::is_same_v; if (!(to_date_or_time - || (std::is_same_v && checkDataType(arguments[0].type.get())))) + || (std::is_same_v && WhichDataType(arguments[0].type).isDateTime()))) { throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1.", @@ -950,7 +950,7 @@ public: + toString(arguments.size()) + ", should be 1 or 2. Second argument (time zone) is optional only make sense for DateTime.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!arguments[0].type->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[0].type)) throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -963,7 +963,7 @@ public: + toString(arguments.size()) + ", should be 1. Second argument makes sense only when converting to DateTime.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!arguments[1].type->isString()) + if (!isString(arguments[1].type)) throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -1020,11 +1020,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!arguments[1].type->isUnsignedInteger()) + if (!isUnsignedInteger(arguments[1].type)) throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN); if (!arguments[1].column) throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN); - if (!arguments[0].type->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[0].type)) throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED); const size_t n = arguments[1].column->getUInt(0); @@ -1140,8 +1140,8 @@ struct ToIntMonotonicity } /// If type is same, too. (Enum has separate case, because it is different data type) - if (checkDataType>(&type) || - checkDataType>(&type)) + if (checkAndGetDataType>(&type) || + checkAndGetDataType>(&type)) return { true, true, true }; /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. @@ -1149,8 +1149,7 @@ struct ToIntMonotonicity return {}; /// If converting from float, for monotonicity, arguments must fit in range of result type. - if (checkDataType(&type) - || checkDataType(&type)) + if (WhichDataType(type).isFloat()) { Float64 left_float = left.get(); Float64 right_float = right.get(); @@ -1460,7 +1459,7 @@ private: static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N) { - if (!from_type->isStringOrFixedString()) + if (!isStringOrFixedString(from_type)) throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED}; return [N] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/) @@ -1469,6 +1468,24 @@ private: }; } + WrapperType createUUIDWrapper(const DataTypePtr & from_type, const DataTypeUUID * const, bool requested_result_is_nullable) const + { + if (requested_result_is_nullable) + throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED}; + + FunctionPtr function = FunctionTo::Type::create(context); + + /// Check conversion using underlying function + { + function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" })); + } + + return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + function->execute(block, arguments, result, input_rows_count); + }; + } + template WrapperType createDecimalWrapper(const DataTypePtr & from_type, const DataTypeDecimal * to_type) const { @@ -1628,7 +1645,7 @@ private: return createStringToEnumWrapper(); else if (checkAndGetDataType(from_type.get())) return createStringToEnumWrapper(); - else if (from_type->isNumber() || from_type->isEnum()) + else if (isNumber(from_type) || isEnum(from_type)) { auto function = Function::create(context); @@ -1878,7 +1895,7 @@ private: { if (from_type->equals(*to_type)) return createIdentityWrapper(from_type); - else if (checkDataType(from_type.get())) + else if (WhichDataType(from_type).isNothing()) return createNothingWrapper(to_type.get()); WrapperType ret; @@ -1920,6 +1937,14 @@ private: ret = createDecimalWrapper(from_type, checkAndGetDataType(to_type.get())); return true; } + if constexpr (std::is_same_v) + { + if (isStringOrFixedString(from_type)) + { + ret = createUUIDWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + return true; + } + } return false; }; @@ -2027,7 +2052,7 @@ private: return monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) return monotonicityForType(type); - else if (from_type->isEnum()) + else if (isEnum(from_type)) { if (const auto type = checkAndGetDataType(to_type)) return monotonicityForType(type); diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index 247d10b3d58..e83dcb60929 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -637,14 +637,14 @@ public: { if (arguments.size() == 1) { - if (!arguments[0].type->isDateOrDateTime()) + if (!isDateOrDateTime(arguments[0].type)) throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else if (arguments.size() == 2) { - if (!checkDataType(arguments[0].type.get()) - || !checkDataType(arguments[1].type.get())) + if (!WhichDataType(arguments[0].type).isDateTime() + || !WhichDataType(arguments[1].type).isString()) throw Exception( "Function " + getName() + " supports 1 or 2 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument (optional) must be " @@ -670,10 +670,11 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); + WhichDataType which(from_type); - if (checkDataType(from_type)) + if (which.isDate()) DateTimeTransformImpl::execute(block, arguments, result, input_rows_count); - else if (checkDataType(from_type)) + else if (which.isDateTime()) DateTimeTransformImpl::execute(block, arguments, result, input_rows_count); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), @@ -945,20 +946,20 @@ public: + toString(arguments.size()) + ", should be 2 or 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!arguments[1].type->isNumber()) + if (!isNumber(arguments[1].type)) throw Exception("Second argument for function " + getName() + " (delta) must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2) { - if (!arguments[0].type->isDateOrDateTime()) + if (!isDateOrDateTime(arguments[0].type)) throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } else { - if (!checkDataType(arguments[0].type.get()) - || !checkDataType(arguments[2].type.get())) + if (!WhichDataType(arguments[0].type).isDateTime() + || !WhichDataType(arguments[2].type).isString()) throw Exception( "Function " + getName() + " supports 2 or 3 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument must be number. " @@ -968,7 +969,7 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - if (checkDataType(arguments[0].type.get())) + if (WhichDataType(arguments[0].type).isDate()) { if (std::is_same_v())), UInt16>) return std::make_shared(); @@ -990,10 +991,11 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); + WhichDataType which(from_type); - if (checkDataType(from_type)) + if (which.isDate()) DateTimeAddIntervalImpl::execute(block, arguments, result); - else if (checkDataType(from_type)) + else if (which.isDateTime()) DateTimeAddIntervalImpl::execute(block, arguments, result); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), @@ -1032,19 +1034,19 @@ public: + toString(arguments.size()) + ", should be 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("First argument for function " + getName() + " (unit) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[1]->isDateOrDateTime()) + if (!isDateOrDateTime(arguments[1])) throw Exception("Second argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[2]->isDateOrDateTime()) + if (!isDateOrDateTime(arguments[2])) throw Exception("Third argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() == 4 && !arguments[3]->isString()) + if (arguments.size() == 4 && !isString(arguments[3])) throw Exception("Fourth argument for function " + getName() + " (timezone) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1296,7 +1298,7 @@ public: + toString(arguments.size()) + ", should be 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!checkDataType(arguments[0].type.get())) + if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -1326,7 +1328,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!checkDataType(arguments[0].get())) + if (!WhichDataType(arguments[0]).isDateTime()) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1453,11 +1455,11 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!checkDataType(arguments[0].get())) + if (!WhichDataType(arguments[0]).isDateTime()) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!checkDataType(arguments[1].get())) + if (!WhichDataType(arguments[1]).isUInt32()) throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index 74f1f91335e..7aa24f5115f 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -82,12 +82,12 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[1].get()) && - !checkDataType(arguments[1].get())) + if (!WhichDataType(arguments[1]).isUInt64() && + !isTuple(arguments[1])) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -230,27 +230,27 @@ private: throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 3 or 4.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - if (!arguments[0]->isString()) + if (!isString(arguments[0])) { throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - if (!arguments[1]->isString()) + if (!isString(arguments[1])) { throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - if (!checkDataType(arguments[2].get()) && - !checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt64() && + !isTuple(arguments[2])) { throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } /// This is for the case of range dictionaries. - if (arguments.size() == 4 && !checkDataType(arguments[3].get())) + if (arguments.size() == 4 && !WhichDataType(arguments[3]).isDate()) { throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() + ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -467,22 +467,22 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[2].get()) && - !checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt64() && + !isTuple(arguments[2])) { throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - if (!arguments[3]->isString()) + if (!isString(arguments[3])) throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() + ", must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -735,20 +735,20 @@ private: if (arguments.size() != 3 && arguments.size() != 4) throw Exception{"Function " + getName() + " takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[2].get()) && - !checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt64() && + !isTuple(arguments[2])) throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (arguments.size() == 4 && !checkDataType(arguments[3].get())) + if (arguments.size() == 4 && !WhichDataType(arguments[3]).isDate()) throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() + ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -1010,20 +1010,20 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[2].get()) && - !checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt64() && + !isTuple(arguments[2])) throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[3].get())) + if (!checkAndGetDataType(arguments[3].get())) throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() + ", must be " + String(DataType{}.getFamilyName()) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -1252,11 +1252,11 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[1].get())) + if (!WhichDataType(arguments[1]).isUInt64()) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -1408,15 +1408,15 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[1].get())) + if (!WhichDataType(arguments[1]).isUInt64()) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!checkDataType(arguments[2].get())) + if (!WhichDataType(arguments[2]).isUInt64()) throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 41585d2bc52..e862794a1da 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -32,7 +32,7 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments throw Exception("Function " + getName() + " expects at least 2 arguments", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION); - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsFindCluster.h b/dbms/src/Functions/FunctionsFindCluster.h index e3f6330817b..dcd76915e8a 100644 --- a/dbms/src/Functions/FunctionsFindCluster.h +++ b/dbms/src/Functions/FunctionsFindCluster.h @@ -116,7 +116,7 @@ public: const auto type_x = arguments[0]; - if (!type_x->isNumber()) + if (!isNumber(type_x)) throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + " must be a numeric type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/dbms/src/Functions/FunctionsFormatting.h b/dbms/src/Functions/FunctionsFormatting.h index adf70607918..d474078ac55 100644 --- a/dbms/src/Functions/FunctionsFormatting.h +++ b/dbms/src/Functions/FunctionsFormatting.h @@ -44,9 +44,9 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - const IDataType * type = arguments[0].get(); + const DataTypePtr & type = arguments[0]; - if (!type->isInteger()) + if (!isInteger(type)) throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); @@ -139,7 +139,7 @@ public: { const IDataType & type = *arguments[0]; - if (!type.isNumber()) + if (!isNumber(type)) throw Exception("Cannot format " + type.getName() + " as size in bytes", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); diff --git a/dbms/src/Functions/FunctionsGeo.cpp b/dbms/src/Functions/FunctionsGeo.cpp index 40e433cf1f4..9841ba03e85 100644 --- a/dbms/src/Functions/FunctionsGeo.cpp +++ b/dbms/src/Functions/FunctionsGeo.cpp @@ -133,7 +133,7 @@ public: for (auto j : ext::range(0, elements.size())) { - if (!elements[j]->isNumber()) + if (!isNumber(elements[j])) { throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -162,7 +162,7 @@ public: const Columns & tuple_columns = tuple_col->getColumns(); const DataTypes & tuple_types = typeid_cast(*block.getByPosition(arguments[0]).type).getElements(); - bool use_float64 = checkDataType(tuple_types[0].get()) || checkDataType(tuple_types[1].get()); + bool use_float64 = WhichDataType(tuple_types[0]).isFloat64() || WhichDataType(tuple_types[1]).isFloat64(); auto & result_column = block.safeGetByPosition(result).column; diff --git a/dbms/src/Functions/FunctionsGeo.h b/dbms/src/Functions/FunctionsGeo.h index ef4ed898076..79e11959e18 100644 --- a/dbms/src/Functions/FunctionsGeo.h +++ b/dbms/src/Functions/FunctionsGeo.h @@ -61,7 +61,7 @@ private: for (const auto arg_idx : ext::range(0, arguments.size())) { const auto arg = arguments[arg_idx].get(); - if (!checkDataType(arg)) + if (!WhichDataType(arg).isFloat64()) throw Exception( "Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -213,7 +213,7 @@ private: for (const auto arg_idx : ext::range(0, arguments.size())) { const auto arg = arguments[arg_idx].get(); - if (!checkDataType(arg)) + if (!WhichDataType(arg).isFloat64()) { throw Exception( "Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64", diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index d10a2bff51e..0bff3451711 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -200,7 +200,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -303,17 +303,18 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); + WhichDataType which(from_type); - if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); - else if (checkDataType(from_type)) executeType(block, arguments, result); + if (which.isUInt8()) executeType(block, arguments, result); + else if (which.isUInt16()) executeType(block, arguments, result); + else if (which.isUInt32()) executeType(block, arguments, result); + else if (which.isUInt64()) executeType(block, arguments, result); + else if (which.isInt8()) executeType(block, arguments, result); + else if (which.isInt16()) executeType(block, arguments, result); + else if (which.isInt32()) executeType(block, arguments, result); + else if (which.isInt64()) executeType(block, arguments, result); + else if (which.isDate()) executeType(block, arguments, result); + else if (which.isDateTime()) executeType(block, arguments, result); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -479,23 +480,25 @@ private: template void executeAny(const IDataType * from_type, const IColumn * icolumn, ColumnUInt64::Container & vec_to) { - if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeString(icolumn, vec_to); - else if (checkDataType(from_type)) executeString(icolumn, vec_to); - else if (checkDataType(from_type)) executeArray(from_type, icolumn, vec_to); + WhichDataType which(from_type); + + if (which.isUInt8()) executeIntType(icolumn, vec_to); + else if (which.isUInt16()) executeIntType(icolumn, vec_to); + else if (which.isUInt32()) executeIntType(icolumn, vec_to); + else if (which.isUInt64()) executeIntType(icolumn, vec_to); + else if (which.isInt8()) executeIntType(icolumn, vec_to); + else if (which.isInt16()) executeIntType(icolumn, vec_to); + else if (which.isInt32()) executeIntType(icolumn, vec_to); + else if (which.isInt64()) executeIntType(icolumn, vec_to); + else if (which.isEnum8()) executeIntType(icolumn, vec_to); + else if (which.isEnum16()) executeIntType(icolumn, vec_to); + else if (which.isDate()) executeIntType(icolumn, vec_to); + else if (which.isDateTime()) executeIntType(icolumn, vec_to); + else if (which.isFloat32()) executeIntType(icolumn, vec_to); + else if (which.isFloat64()) executeIntType(icolumn, vec_to); + else if (which.isString()) executeString(icolumn, vec_to); + else if (which.isFixedString()) executeString(icolumn, vec_to); + else if (which.isArray()) executeArray(from_type, icolumn, vec_to); else throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -602,23 +605,23 @@ public: const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]); const IDataType * from_type = col.type.get(); const IColumn * icolumn = col.column.get(); + WhichDataType which(from_type); - if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeIntType(icolumn, vec_to); - else if (checkDataType(from_type)) executeString(icolumn, vec_to); - else if (checkDataType(from_type)) executeString(icolumn, vec_to); + if (which.isUInt8()) executeIntType(icolumn, vec_to); + else if (which.isUInt16()) executeIntType(icolumn, vec_to); + else if (which.isUInt32()) executeIntType(icolumn, vec_to); + else if (which.isUInt64()) executeIntType(icolumn, vec_to); + else if (which.isInt8()) executeIntType(icolumn, vec_to); + else if (which.isInt16()) executeIntType(icolumn, vec_to); + else if (which.isInt32()) executeIntType(icolumn, vec_to); + else if (which.isInt64()) executeIntType(icolumn, vec_to); + else if (which.isEnum8()) executeIntType(icolumn, vec_to); + else if (which.isEnum16()) executeIntType(icolumn, vec_to); + else if (which.isDate()) executeIntType(icolumn, vec_to); + else if (which.isDateTime()) executeIntType(icolumn, vec_to); + else if (which.isFloat32()) executeIntType(icolumn, vec_to); + else if (which.isFloat64()) executeIntType(icolumn, vec_to); + else if (which.isStringOrFixedString()) executeString(icolumn, vec_to); else throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -843,13 +846,13 @@ public: toString(arg_count) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto first_arg = arguments.front().get(); - if (!checkDataType(first_arg)) + if (!WhichDataType(first_arg).isString()) throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (arg_count == 2) { - const auto second_arg = arguments.back().get(); - if (!second_arg->isInteger()) + const auto & second_arg = arguments.back(); + if (!isInteger(second_arg)) throw Exception{"Illegal type " + second_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } diff --git a/dbms/src/Functions/FunctionsHigherOrder.h b/dbms/src/Functions/FunctionsHigherOrder.h index 1f564bcba46..f292eaf8ce7 100644 --- a/dbms/src/Functions/FunctionsHigherOrder.h +++ b/dbms/src/Functions/FunctionsHigherOrder.h @@ -310,20 +310,15 @@ struct ArraySumImpl static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & /*array_element*/) { - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + WhichDataType which(expression_return); + + if (which.isNativeUInt()) return std::make_shared(); - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + if (which.isNativeInt()) return std::make_shared(); - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + if (which.isFloat()) return std::make_shared(); throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -602,20 +597,15 @@ struct ArrayCumSumImpl static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & /*array_element*/) { - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + WhichDataType which(expression_return); + + if (which.isNativeUInt()) return std::make_shared(std::make_shared()); - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + if (which.isNativeInt()) return std::make_shared(std::make_shared()); - if (checkDataType(&*expression_return) || - checkDataType(&*expression_return)) + if (which.isFloat()) return std::make_shared(std::make_shared()); throw Exception("arrayCumSum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -824,7 +814,7 @@ public: DataTypePtr nested_type = array_type->getNestedType(); - if (Impl::needBoolean() && !checkDataType(&*nested_type)) + if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8()) throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -845,7 +835,7 @@ public: /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. DataTypePtr return_type = data_type_function->getReturnType(); - if (Impl::needBoolean() && !checkDataType(&*return_type)) + if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsLogical.h b/dbms/src/Functions/FunctionsLogical.h index a56a37dbf6c..6f2bb62f9cf 100644 --- a/dbms/src/Functions/FunctionsLogical.h +++ b/dbms/src/Functions/FunctionsLogical.h @@ -309,8 +309,8 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (size_t i = 0; i < arguments.size(); ++i) - if (!(arguments[i]->isNumber() - || (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || removeNullable(arguments[i])->isNumber())))) + if (!(isNumber(arguments[i]) + || (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || isNumber(removeNullable(arguments[i])))))) throw Exception("Illegal type (" + arguments[i]->getName() + ") of " + toString(i + 1) + " argument of function " + getName(), @@ -488,7 +488,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isNumber()) + if (!isNumber(arguments[0])) throw Exception("Illegal type (" + arguments[0]->getName() + ") of argument of function " + getName(), diff --git a/dbms/src/Functions/FunctionsMath.h b/dbms/src/Functions/FunctionsMath.h index 59aee27ab06..fc3ed647d05 100644 --- a/dbms/src/Functions/FunctionsMath.h +++ b/dbms/src/Functions/FunctionsMath.h @@ -78,7 +78,7 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments.front()->isNumber()) + if (!isNumber(arguments.front())) throw Exception{"Illegal type " + arguments.front()->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return std::make_shared(); @@ -199,7 +199,7 @@ private: { const auto check_argument_type = [this] (const IDataType * arg) { - if (!arg->isNumber()) + if (!isNumber(arg)) throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; }; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp deleted file mode 100644 index a4f7da52e40..00000000000 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ /dev/null @@ -1,2036 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int FUNCTION_IS_SPECIAL; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int TOO_SLOW; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO; -} - -/** Helper functions - * - * visibleWidth(x) - calculates the approximate width when outputting the value in a text (tab-separated) form to the console. - * - * toTypeName(x) - get the type name - * blockSize() - get the block size - * materialize(x) - materialize the constant - * ignore(...) is a function that takes any arguments, and always returns 0. - * sleep(seconds) - the specified number of seconds sleeps each block. - * - * in(x, set) - function for evaluating the IN - * notIn(x, set) - and NOT IN. - * - * arrayJoin(arr) - a special function - it can not be executed directly; - * is used only to get the result type of the corresponding expression. - * - * replicate(x, arr) - creates an array of the same size as arr, all elements of which are equal to x; - * for example: replicate(1, ['a', 'b', 'c']) = [1, 1, 1]. - * - * sleep(n) - sleeps n seconds for each block. - * - * bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max. - * - * version() - returns the current version of the server on the line. - * - * finalizeAggregation(agg_state) - get the result from the aggregation state. - * - * runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values, - * are the result of the accumulation of these states for a set of block lines, from the first to the current line. - */ - - -class FunctionCurrentDatabase : public IFunction -{ - const String db_name; - -public: - static constexpr auto name = "currentDatabase"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getCurrentDatabase()); - } - - explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} - { - } - - String getName() const override - { - return name; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name); - } -}; - - -/// Get the host name. Is is constant on single server, but is not constant in distributed queries. -class FunctionHostName : public IFunction -{ -public: - static constexpr auto name = "hostName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - /** convertToFullColumn needed because in distributed query processing, - * each server returns its own value. - */ - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( - input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst(); - } -}; - - -class FunctionVisibleWidth : public IFunction -{ -public: - static constexpr auto name = "visibleWidth"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool useDefaultImplementationForNulls() const override { return false; } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - /// Execute the function on the block. - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override; -}; - - -/// Returns name of IDataType instance (name of data type). -class FunctionToTypeName : public IFunction -{ -public: - static constexpr auto name = "toTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - /// Execute the function on the block. - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).type->getName()); - } -}; - - -/// Returns number of fields in Enum data type of passed value. -class FunctionGetSizeOfEnumType : public IFunction -{ -public: - static constexpr auto name = "getSizeOfEnumType"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (checkDataType(arguments[0].get())) - return std::make_shared(); - else if (checkDataType(arguments[0].get())) - return std::make_shared(); - - throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(type->getValues().size())); - else if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, UInt64(type->getValues().size())); - else - throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } -}; - - - -/// Returns name of IColumn instance. -class FunctionToColumnTypeName : public IFunction -{ -public: - static constexpr auto name = "toColumnTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); - } -}; - - - -/// Dump the structure of type and column. -class FunctionDumpColumnStructure : public IFunction -{ -public: - static constexpr auto name = "dumpColumnStructure"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const auto & elem = block.getByPosition(arguments[0]); - - /// Note that the result is not a constant, because it contains block size. - - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, - elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst(); - } -}; - - -/// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String). -class FunctionDefaultValueOfArgumentType : public IFunction -{ -public: - static constexpr auto name = "defaultValueOfArgumentType"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments[0]; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const IDataType & type = *block.getByPosition(arguments[0]).type; - block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); - } -}; - - -class FunctionBlockSize : public IFunction -{ -public: - static constexpr auto name = "blockSize"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count); - } -}; - - -class FunctionRowNumberInBlock : public IFunction -{ -public: - static constexpr auto name = "rowNumberInBlock"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - auto column = ColumnUInt64::create(); - auto & data = column->getData(); - data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = i; - - block.getByPosition(result).column = std::move(column); - } -}; - - -/** Incremental block number among calls of this function. */ -class FunctionBlockNumber : public IFunction -{ -private: - std::atomic block_number{0}; - -public: - static constexpr auto name = "blockNumber"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - size_t current_block_number = block_number++; - block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number); - } -}; - - -/** Incremental number of row within all blocks passed to this function. */ -class FunctionRowNumberInAllBlocks : public IFunction -{ -private: - std::atomic rows{0}; - -public: - static constexpr auto name = "rowNumberInAllBlocks"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - size_t current_row_number = rows.fetch_add(input_rows_count); - - auto column = ColumnUInt64::create(); - auto & data = column->getData(); - data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = current_row_number + i; - - block.getByPosition(result).column = std::move(column); - } -}; - - -enum class FunctionSleepVariant -{ - PerBlock, - PerRow -}; - -template -class FunctionSleep : public IFunction -{ -public: - static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; - static FunctionPtr create(const Context &) - { - return std::make_shared>(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - /// Do not sleep during query analysis. - bool isSuitableForConstantFolding() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!checkDataType(arguments[0].get()) && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get()) - && !checkDataType(arguments[0].get())) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const IColumn * col = block.getByPosition(arguments[0]).column.get(); - - if (!col->isColumnConst()) - throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); - - Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), static_cast(*col).getField()); - - if (seconds < 0) - throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS); - - size_t size = col->size(); - - /// We do not sleep if the block is empty. - if (size > 0) - { - unsigned useconds = seconds * (variant == FunctionSleepVariant::PerBlock ? 1 : size) * 1e6; - - /// When sleeping, the query cannot be cancelled. For abitily to cancel query, we limit sleep time. - if (useconds > 3000000) /// The choice is arbitary - throw Exception("The maximum sleep time is 3000000 microseconds. Requested: " + toString(useconds), ErrorCodes::TOO_SLOW); - - usleep(useconds); - } - - /// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each block. - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, UInt64(0))->convertToFullColumnIfConst(); - } -}; - - -class FunctionMaterialize : public IFunction -{ -public: - static constexpr auto name = "materialize"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments[0]; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto & src = block.getByPosition(arguments[0]).column; - if (ColumnPtr converted = src->convertToFullColumnIfConst()) - block.getByPosition(result).column = converted; - else - block.getByPosition(result).column = src; - } -}; - -template -struct FunctionInName; -template <> -struct FunctionInName -{ - static constexpr auto name = "in"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "globalIn"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "notIn"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "globalNotIn"; -}; - -template -class FunctionIn : public IFunction -{ -public: - static constexpr auto name = FunctionInName::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool useDefaultImplementationForNulls() const override - { - return false; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - /// Second argument must be ColumnSet. - ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; - const ColumnSet * column_set = typeid_cast(&*column_set_ptr); - if (!column_set) - throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), - ErrorCodes::ILLEGAL_COLUMN); - - Block block_of_key_columns; - - /// First argument may be tuple or single column. - const ColumnWithTypeAndName & left_arg = block.getByPosition(arguments[0]); - const ColumnTuple * tuple = typeid_cast(left_arg.column.get()); - const ColumnConst * const_tuple = checkAndGetColumnConst(left_arg.column.get()); - const DataTypeTuple * type_tuple = typeid_cast(left_arg.type.get()); - - ColumnPtr materialized_tuple; - if (const_tuple) - { - materialized_tuple = const_tuple->convertToFullColumn(); - tuple = typeid_cast(materialized_tuple.get()); - } - - auto set = column_set->getData(); - auto set_types = set->getDataTypes(); - if (tuple && (set_types.size() != 1 || !set_types[0]->equals(*type_tuple))) - { - const Columns & tuple_columns = tuple->getColumns(); - const DataTypes & tuple_types = type_tuple->getElements(); - size_t tuple_size = tuple_columns.size(); - for (size_t i = 0; i < tuple_size; ++i) - block_of_key_columns.insert({ tuple_columns[i], tuple_types[i], "" }); - } - else - block_of_key_columns.insert(left_arg); - - block.getByPosition(result).column = set->execute(block_of_key_columns, negative); - } -}; - - -class FunctionIgnore : public IFunction -{ -public: - static constexpr auto name = "ignore"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - String getName() const override - { - return name; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); - } -}; - - -/** The `indexHint` function takes any number of any arguments and always returns one. - * - * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) - * - the expressions inside it are not evaluated; - * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, - * as if instead of using it the expression itself would be. - * - * Example: WHERE something AND indexHint(CounterID = 34) - * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. - * - * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. - */ -class FunctionIndexHint : public IFunction -{ -public: - static constexpr auto name = "indexHint"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - String getName() const override - { - return name; - } - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(1)); - } -}; - - -class FunctionIdentity : public IFunction -{ -public: - static constexpr auto name = "identity"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments.front(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - block.getByPosition(result).column = block.getByPosition(arguments.front()).column; - } -}; - - -class FunctionArrayJoin : public IFunction -{ -public: - static constexpr auto name = "arrayJoin"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - /** It could return many different values for single argument. */ - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeArray * arr = checkAndGetDataType(arguments[0].get()); - if (!arr) - throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arr->getNestedType(); - } - - void executeImpl(Block &, const ColumnNumbers &, size_t, size_t /*input_rows_count*/) override - { - throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL); - } - - /// Because of function cannot be executed directly. - bool isSuitableForConstantFolding() const override - { - return false; - } -}; - - -FunctionPtr FunctionReplicate::create(const Context &) -{ - return std::make_shared(); -} - -DataTypePtr FunctionReplicate::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type) - throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(arguments[0]); -} - -void FunctionReplicate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - ColumnPtr first_column = block.getByPosition(arguments[0]).column; - - const ColumnArray * array_column = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); - ColumnPtr temp_column; - - if (!array_column) - { - auto const_array_column = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); - if (!const_array_column) - throw Exception("Unexpected column for replicate", ErrorCodes::ILLEGAL_COLUMN); - temp_column = const_array_column->convertToFullColumn(); - array_column = checkAndGetColumn(temp_column.get()); - } - - block.getByPosition(result).column - = ColumnArray::create(first_column->replicate(array_column->getOffsets()), array_column->getOffsetsPtr()); -} - -/** Returns a string with nice Unicode-art bar with resolution of 1/8 part of symbol. - */ -class FunctionBar : public IFunction -{ -public: - static constexpr auto name = "bar"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() != 3 && arguments.size() != 4) - throw Exception("Function " + getName() - + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed " - + toString(arguments.size()) - + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!arguments[0]->isNumber() || !arguments[1]->isNumber() || !arguments[2]->isNumber() - || (arguments.size() == 4 && !arguments[3]->isNumber())) - throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - Int64 min = extractConstant(block, arguments, 1, "Second"); /// The level at which the line has zero length. - Int64 max = extractConstant(block, arguments, 2, "Third"); /// The level at which the line has the maximum length. - - /// The maximum width of the bar in characters, by default. - Float64 max_width = arguments.size() == 4 ? extractConstant(block, arguments, 3, "Fourth") : 80; - - if (max_width < 1) - throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (max_width > 1000) - throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - const auto & src = *block.getByPosition(arguments[0]).column; - - auto res_column = ColumnString::create(); - - if (executeNumber(src, *res_column, min, max, max_width) || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width)) - { - block.getByPosition(result).column = std::move(res_column); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } - -private: - template - T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const - { - const auto & column = *block.getByPosition(arguments[argument_pos]).column; - - if (!column.isColumnConst()) - throw Exception( - which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); - - return applyVisitor(FieldVisitorConvertToNumber(), column[0]); - } - - template - static void fill(const PaddedPODArray & src, - ColumnString::Chars_t & dst_chars, - ColumnString::Offsets & dst_offsets, - Int64 min, - Int64 max, - Float64 max_width) - { - size_t size = src.size(); - size_t current_offset = 0; - - dst_offsets.resize(size); - dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated. - - for (size_t i = 0; i < size; ++i) - { - Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width); - size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; - dst_chars.resize(next_size); - UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset])); - current_offset = next_size; - dst_offsets[i] = current_offset; - } - } - - template - static void fill(T src, String & dst_chars, Int64 min, Int64 max, Float64 max_width) - { - Float64 width = UnicodeBar::getWidth(src, min, max, max_width); - dst_chars.resize(UnicodeBar::getWidthInBytes(width)); - UnicodeBar::render(width, dst_chars.data()); - } - - template - static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width) - { - if (const ColumnVector * col = checkAndGetColumn>(&src)) - { - fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width); - return true; - } - else - return false; - } -}; - - -template -class FunctionNumericPredicate : public IFunction -{ -public: - static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments.front()->isNumber()) - throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto in = block.getByPosition(arguments.front()).column.get(); - - if ( !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result)) - throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - - template - bool execute(Block & block, const IColumn * in_untyped, const size_t result) - { - if (const auto in = checkAndGetColumn>(in_untyped)) - { - const auto size = in->size(); - - auto out = ColumnUInt8::create(size); - - const auto & in_data = in->getData(); - auto & out_data = out->getData(); - - for (const auto i : ext::range(0, size)) - out_data[i] = Impl::execute(in_data[i]); - - block.getByPosition(result).column = std::move(out); - return true; - } - - return false; - } -}; - -struct IsFiniteImpl -{ - /// Better implementation, because isinf, isfinite, isnan are not inlined for unknown reason. - /// Assuming IEEE 754. - /// NOTE gcc 7 doesn't vectorize this loop. - - static constexpr auto name = "isFinite"; - template - static bool execute(const T t) - { - if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b01111111100000000000000000000000) - != 0b01111111100000000000000000000000; - else if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b0111111111110000000000000000000000000000000000000000000000000000) - != 0b0111111111110000000000000000000000000000000000000000000000000000; - else - { - (void)t; - return true; - } - } -}; - -struct IsInfiniteImpl -{ - static constexpr auto name = "isInfinite"; - template - static bool execute(const T t) - { - if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b01111111111111111111111111111111) - == 0b01111111100000000000000000000000; - else if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b0111111111111111111111111111111111111111111111111111111111111111) - == 0b0111111111110000000000000000000000000000000000000000000000000000; - else - { - (void)t; - return false; - } - } -}; - -struct IsNaNImpl -{ - static constexpr auto name = "isNaN"; - template - static bool execute(const T t) - { - return t != t; //-V501 - } -}; - -using FunctionIsFinite = FunctionNumericPredicate; -using FunctionIsInfinite = FunctionNumericPredicate; -using FunctionIsNaN = FunctionNumericPredicate; - - -/** Returns server version (constant). - */ -class FunctionVersion : public IFunction -{ -public: - static constexpr auto name = "version"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - static const std::string version = getVersion(); - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, version); - } - -private: - std::string getVersion() const; -}; - - -/** Returns server uptime in seconds. - */ -class FunctionUptime : public IFunction -{ -public: - static constexpr auto name = "uptime"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getUptimeSeconds()); - } - - explicit FunctionUptime(time_t uptime_) : uptime(uptime_) - { - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt32().createColumnConst(input_rows_count, static_cast(uptime)); - } - -private: - time_t uptime; -}; - - -/** Returns the server time zone. - */ -class FunctionTimeZone : public IFunction -{ -public: - static constexpr auto name = "timezone"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone()); - } -}; - - -/** Quite unusual function. - * Takes state of aggregate function (example runningAccumulate(uniqState(UserID))), - * and for each row of block, return result of aggregate function on merge of states of all previous rows and current row. - * - * So, result of function depends on partition of data to blocks and on order of data in block. - */ -class FunctionRunningAccumulate : public IFunction -{ -public: - static constexpr auto name = "runningAccumulate"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); - if (!type) - throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnAggregateFunction * column_with_states - = typeid_cast(&*block.getByPosition(arguments.at(0)).column); - if (!column_with_states) - throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() - + " of first argument of function " - + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction(); - const IAggregateFunction & agg_func = *aggregate_function_ptr; - - AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData()); - agg_func.create(place.data()); - SCOPE_EXIT(agg_func.destroy(place.data())); - - std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; - - auto result_column_ptr = agg_func.getReturnType()->createColumn(); - IColumn & result_column = *result_column_ptr; - result_column.reserve(column_with_states->size()); - - const auto & states = column_with_states->getData(); - for (const auto & state_to_add : states) - { - /// Will pass empty arena if agg_func does not allocate memory in arena - agg_func.merge(place.data(), state_to_add, arena.get()); - agg_func.insertResultInto(place.data(), result_column); - } - - block.getByPosition(result).column = std::move(result_column_ptr); - } -}; - -template -struct FunctionRunningDifferenceName; - -template <> -struct FunctionRunningDifferenceName -{ - static constexpr auto name = "runningDifference"; -}; - -template <> -struct FunctionRunningDifferenceName -{ - static constexpr auto name = "runningDifferenceStartingWithFirstValue"; -}; - -/** Calculate difference of consecutive values in block. - * So, result of function depends on partition of data to blocks and on order of data in block. - */ -template -class FunctionRunningDifferenceImpl : public IFunction -{ -private: - /// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented. - - template - static void process(const PaddedPODArray & src, PaddedPODArray & dst, const NullMap * null_map) - { - size_t size = src.size(); - dst.resize(size); - - if (size == 0) - return; - - /// It is possible to SIMD optimize this loop. By no need for that in practice. - - Src prev; - bool has_prev_value = false; - - for (size_t i = 0; i < size; ++i) - { - if (null_map && (*null_map)[i]) - continue; - - if (!has_prev_value) - { - dst[i] = is_first_line_zero ? 0 : src[i]; - prev = src[i]; - has_prev_value = true; - } - else - { - auto cur = src[i]; - dst[i] = static_cast(cur) - prev; - prev = cur; - } - } - } - - /// Result type is same as result of subtraction of argument types. - template - using DstFieldType = typename NumberTraits::ResultOfSubtraction::Type; - - /// Call polymorphic lambda with tag argument of concrete field type of src_type. - template - void dispatchForSourceType(const IDataType & src_type, F && f) const - { - if (checkDataType(&src_type)) - f(UInt8()); - else if (checkDataType(&src_type)) - f(UInt16()); - else if (checkDataType(&src_type)) - f(UInt32()); - else if (checkDataType(&src_type)) - f(UInt64()); - else if (checkDataType(&src_type)) - f(Int8()); - else if (checkDataType(&src_type)) - f(Int16()); - else if (checkDataType(&src_type)) - f(Int32()); - else if (checkDataType(&src_type)) - f(Int64()); - else if (checkDataType(&src_type)) - f(Float32()); - else if (checkDataType(&src_type)) - f(Float64()); - else if (checkDataType(&src_type)) - f(DataTypeDate::FieldType()); - else if (checkDataType(&src_type)) - f(DataTypeDateTime::FieldType()); - else - throw Exception("Argument for function " + getName() + " must have numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - -public: - static constexpr auto name = FunctionRunningDifferenceName::name; - - static FunctionPtr create(const Context &) - { - return std::make_shared>(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - DataTypePtr res; - dispatchForSourceType(*removeNullable(arguments[0]), [&](auto field_type_tag) - { - res = std::make_shared>>(); - }); - - if (arguments[0]->isNullable()) - res = makeNullable(res); - - return res; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - auto & src = block.getByPosition(arguments.at(0)); - const auto & res_type = block.getByPosition(result).type; - - /// When column is constant, its difference is zero. - if (src.column->isColumnConst()) - { - block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto res_column = removeNullable(res_type)->createColumn(); - auto * src_column = src.column.get(); - ColumnPtr null_map_column = nullptr; - const NullMap * null_map = nullptr; - if (auto * nullable_column = checkAndGetColumn(src_column)) - { - src_column = &nullable_column->getNestedColumn(); - null_map_column = nullable_column->getNullMapColumnPtr(); - null_map = &nullable_column->getNullMapData(); - } - - dispatchForSourceType(*removeNullable(src.type), [&](auto field_type_tag) - { - using SrcFieldType = decltype(field_type_tag); - - process(static_cast &>(*src_column).getData(), - static_cast> &>(*res_column).getData(), null_map); - }); - - if (null_map_column) - block.getByPosition(result).column = ColumnNullable::create(std::move(res_column), null_map_column); - else - block.getByPosition(result).column = std::move(res_column); - } -}; - -using FunctionRunningDifference = FunctionRunningDifferenceImpl; -using FunctionRunningIncome = FunctionRunningDifferenceImpl; - - -/** Takes state of aggregate function. Returns result of aggregation (finalized state). - */ -class FunctionFinalizeAggregation : public IFunction -{ -public: - static constexpr auto name = "finalizeAggregation"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); - if (!type) - throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnAggregateFunction * column_with_states - = typeid_cast(&*block.getByPosition(arguments.at(0)).column); - if (!column_with_states) - throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() - + " of first argument of function " - + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - block.getByPosition(result).column = column_with_states->convertToValues(); - } -}; - - -/** Usage: - * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') - */ -class FunctionHasColumnInTable : public IFunction -{ -public: - static constexpr auto name = "hasColumnInTable"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getGlobalContext()); - } - - explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) - { - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - String getName() const override - { - return name; - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - const Context & global_context; -}; - - -void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto & src = block.getByPosition(arguments[0]); - size_t size = input_rows_count; - - auto res_col = ColumnUInt64::create(size); - auto & res_data = static_cast(*res_col).getData(); - - /// For simplicity reasons, function is implemented by serializing into temporary buffer. - - String tmp; - FormatSettings format_settings; - for (size_t i = 0; i < size; ++i) - { - { - WriteBufferFromString out(tmp); - src.type->serializeText(*src.column, i, out, format_settings); - } - - res_data[i] = UTF8::countCodePoints(reinterpret_cast(tmp.data()), tmp.size()); - } - - block.getByPosition(result).column = std::move(res_col); -} - - -DataTypePtr FunctionHasColumnInTable::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const -{ - if (arguments.size() < 3 || arguments.size() > 6) - throw Exception{"Invalid number of arguments for function " + getName(), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - static const std::string arg_pos_description[] = {"First", "Second", "Third", "Fourth", "Fifth", "Sixth"}; - for (size_t i = 0; i < arguments.size(); ++i) - { - const ColumnWithTypeAndName & argument = arguments[i]; - - if (!checkColumnConst(argument.column.get())) - { - throw Exception(arg_pos_description[i] + " argument for function " + getName() + " must be const String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - - return std::make_shared(); -} - - -void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto get_string_from_block = [&](size_t column_pos) -> String - { - ColumnPtr column = block.getByPosition(column_pos).column; - const ColumnConst * const_column = checkAndGetColumnConst(column.get()); - return const_column->getValue(); - }; - - size_t arg = 0; - String host_name; - String user_name; - String password; - - if (arguments.size() > 3) - host_name = get_string_from_block(arguments[arg++]); - - if (arguments.size() > 4) - user_name = get_string_from_block(arguments[arg++]); - - if (arguments.size() > 5) - password = get_string_from_block(arguments[arg++]); - - String database_name = get_string_from_block(arguments[arg++]); - String table_name = get_string_from_block(arguments[arg++]); - String column_name = get_string_from_block(arguments[arg++]); - - bool has_column; - if (host_name.empty()) - { - const StoragePtr & table = global_context.getTable(database_name, table_name); - has_column = table->hasColumn(column_name); - } - else - { - std::vector> host_names = {{ host_name }}; - auto cluster = std::make_shared(global_context.getSettings(), host_names, !user_name.empty() ? user_name : "default", password, global_context.getTCPPort(), false); - auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context); - has_column = remote_columns.hasPhysical(column_name); - } - - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(has_column)); -} - - -/// Throw an exception if the argument is non zero. -class FunctionThrowIf : public IFunction -{ -public: - static constexpr auto name = "throwIf"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments.front()->isNumber()) - throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto in = block.getByPosition(arguments.front()).column.get(); - - if ( !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result)) - throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - - template - bool execute(Block & block, const IColumn * in_untyped, const size_t result) - { - if (const auto in = checkAndGetColumn>(in_untyped)) - { - const auto & in_data = in->getData(); - if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) - throw Exception("Value passed to 'throwIf' function is non zero", ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO); - - /// We return non constant to avoid constant folding. - block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0); - return true; - } - - return false; - } -}; - - -class FunctionToLowCardinality: public IFunction -{ -public: - static constexpr auto name = "toLowCardinality"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments[0]->withDictionary()) - return arguments[0]; - - return std::make_shared(arguments[0]); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - - if (arg.type->withDictionary()) - res.column = arg.column; - else - { - auto column = res.type->createColumn(); - typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); - res.column = std::move(column); - } - } -}; - -class FunctionLowCardinalityIndexes: public IFunction -{ -public: - static constexpr auto name = "lowCardinalityIndexes"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); - auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); - auto & data = new_indexes_col->getData(); - for (size_t i = 0; i < data.size(); ++i) - data[i] = indexes_col->getUInt(i); - - res.column = std::move(new_indexes_col); - } -}; - -class FunctionLowCardinalityKeys: public IFunction -{ -public: - static constexpr auto name = "lowCardinalityKeys"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getDictionaryType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - const auto * column_with_dictionary = typeid_cast(arg.column.get()); - res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); - } -}; - - -std::string FunctionVersion::getVersion() const -{ - return VERSION_STRING; -} - - -void registerFunctionsMiscellaneous(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} -} diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index e97457e3e13..07fe04389cf 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -11,32 +11,6 @@ namespace DB { -/** Creates an array, multiplying the column (the first argument) by the number of elements in the array (the second argument). - */ -class FunctionReplicate : public IFunction -{ -public: - static constexpr auto name = "replicate"; - static FunctionPtr create(const Context & context); - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - - /// Executes expression. Uses for lambda functions implementation. Can't be created from factory. class FunctionExpression : public IFunctionBase, public IPreparedFunction, public std::enable_shared_from_this diff --git a/dbms/src/Functions/FunctionsNull.cpp b/dbms/src/Functions/FunctionsNull.cpp deleted file mode 100644 index d44e1152e3c..00000000000 --- a/dbms/src/Functions/FunctionsNull.cpp +++ /dev/null @@ -1,390 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -void registerFunctionsNull(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(); -} - -/// Implementation of isNull. - -FunctionPtr FunctionIsNull::create(const Context &) -{ - return std::make_shared(); -} - -std::string FunctionIsNull::getName() const -{ - return name; -} - -DataTypePtr FunctionIsNull::getReturnTypeImpl(const DataTypes &) const -{ - return std::make_shared(); -} - -void FunctionIsNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const ColumnWithTypeAndName & elem = block.getByPosition(arguments[0]); - if (elem.column->isColumnNullable()) - { - /// Merely return the embedded null map. - block.getByPosition(result).column = static_cast(*elem.column).getNullMapColumnPtr(); - } - else - { - /// Since no element is nullable, return a zero-constant column representing - /// a zero-filled null map. - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(elem.column->size(), UInt64(0)); - } -} - -/// Implementation of isNotNull. - -FunctionPtr FunctionIsNotNull::create(const Context &) -{ - return std::make_shared(); -} - -std::string FunctionIsNotNull::getName() const -{ - return name; -} - -DataTypePtr FunctionIsNotNull::getReturnTypeImpl(const DataTypes &) const -{ - return std::make_shared(); -} - -void FunctionIsNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - Block temp_block - { - block.getByPosition(arguments[0]), - { - nullptr, - std::make_shared(), - "" - }, - { - nullptr, - std::make_shared(), - "" - } - }; - - FunctionIsNull{}.execute(temp_block, {0}, 1, input_rows_count); - FunctionNot{}.execute(temp_block, {1}, 2, input_rows_count); - - block.getByPosition(result).column = std::move(temp_block.getByPosition(2).column); -} - -/// Implementation of coalesce. - -FunctionPtr FunctionCoalesce::create(const Context & context) -{ - return std::make_shared(context); -} - -std::string FunctionCoalesce::getName() const -{ - return name; -} - -DataTypePtr FunctionCoalesce::getReturnTypeImpl(const DataTypes & arguments) const -{ - /// Skip all NULL arguments. If any argument is non-Nullable, skip all next arguments. - DataTypes filtered_args; - filtered_args.reserve(arguments.size()); - for (const auto & arg : arguments) - { - if (arg->onlyNull()) - continue; - - filtered_args.push_back(arg); - - if (!arg->isNullable()) - break; - } - - DataTypes new_args; - for (size_t i = 0; i < filtered_args.size(); ++i) - { - bool is_last = i + 1 == filtered_args.size(); - - if (is_last) - { - new_args.push_back(filtered_args[i]); - } - else - { - new_args.push_back(std::make_shared()); - new_args.push_back(removeNullable(filtered_args[i])); - } - } - - if (new_args.empty()) - return std::make_shared(std::make_shared()); - if (new_args.size() == 1) - return new_args.front(); - - auto res = FunctionMultiIf{context}.getReturnTypeImpl(new_args); - - /// if last argument is not nullable, result should be also not nullable - if (!new_args.back()->isNullable() && res->isNullable()) - res = removeNullable(res); - - return res; -} - -void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - /// coalesce(arg0, arg1, ..., argN) is essentially - /// multiIf(isNotNull(arg0), assumeNotNull(arg0), isNotNull(arg1), assumeNotNull(arg1), ..., argN) - /// with constant NULL arguments removed. - - ColumnNumbers filtered_args; - filtered_args.reserve(arguments.size()); - for (const auto & arg : arguments) - { - const auto & type = block.getByPosition(arg).type; - - if (type->onlyNull()) - continue; - - filtered_args.push_back(arg); - - if (!type->isNullable()) - break; - } - - FunctionIsNotNull is_not_null; - FunctionAssumeNotNull assume_not_null; - ColumnNumbers multi_if_args; - - Block temp_block = block; - - for (size_t i = 0; i < filtered_args.size(); ++i) - { - size_t res_pos = temp_block.columns(); - bool is_last = i + 1 == filtered_args.size(); - - if (is_last) - { - multi_if_args.push_back(filtered_args[i]); - } - else - { - temp_block.insert({nullptr, std::make_shared(), ""}); - is_not_null.execute(temp_block, {filtered_args[i]}, res_pos, input_rows_count); - temp_block.insert({nullptr, removeNullable(block.getByPosition(filtered_args[i]).type), ""}); - assume_not_null.execute(temp_block, {filtered_args[i]}, res_pos + 1, input_rows_count); - - multi_if_args.push_back(res_pos); - multi_if_args.push_back(res_pos + 1); - } - } - - /// If all arguments appeared to be NULL. - if (multi_if_args.empty()) - { - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - if (multi_if_args.size() == 1) - { - block.getByPosition(result).column = block.getByPosition(multi_if_args.front()).column; - return; - } - - FunctionMultiIf{context}.execute(temp_block, multi_if_args, result, input_rows_count); - - ColumnPtr res = std::move(temp_block.getByPosition(result).column); - - /// if last argument is not nullable, result should be also not nullable - if (!block.getByPosition(multi_if_args.back()).column->isColumnNullable() && res->isColumnNullable()) - res = static_cast(*res).getNestedColumnPtr(); - - block.getByPosition(result).column = std::move(res); -} - -/// Implementation of ifNull. - -FunctionPtr FunctionIfNull::create(const Context &) -{ - return std::make_shared(); -} - -std::string FunctionIfNull::getName() const -{ - return name; -} - -DataTypePtr FunctionIfNull::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments[0]->onlyNull()) - return arguments[1]; - - if (!arguments[0]->isNullable()) - return arguments[0]; - - return FunctionIf{}.getReturnTypeImpl({std::make_shared(), removeNullable(arguments[0]), arguments[1]}); -} - -void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - /// Always null. - if (block.getByPosition(arguments[0]).type->onlyNull()) - { - block.getByPosition(result).column = block.getByPosition(arguments[1]).column; - return; - } - - /// Could not contain nulls, so nullIf makes no sense. - if (!block.getByPosition(arguments[0]).type->isNullable()) - { - block.getByPosition(result).column = block.getByPosition(arguments[0]).column; - return; - } - - /// ifNull(col1, col2) == if(isNotNull(col1), assumeNotNull(col1), col2) - - Block temp_block = block; - - size_t is_not_null_pos = temp_block.columns(); - temp_block.insert({nullptr, std::make_shared(), ""}); - size_t assume_not_null_pos = temp_block.columns(); - temp_block.insert({nullptr, removeNullable(block.getByPosition(arguments[0]).type), ""}); - - FunctionIsNotNull{}.execute(temp_block, {arguments[0]}, is_not_null_pos, input_rows_count); - FunctionAssumeNotNull{}.execute(temp_block, {arguments[0]}, assume_not_null_pos, input_rows_count); - - FunctionIf{}.execute(temp_block, {is_not_null_pos, assume_not_null_pos, arguments[1]}, result, input_rows_count); - - block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); -} - -/// Implementation of nullIf. - -FunctionPtr FunctionNullIf::create(const Context & context) -{ - return std::make_shared(context); -} - -FunctionNullIf::FunctionNullIf(const Context & context) : context(context) {} - -std::string FunctionNullIf::getName() const -{ - return name; -} - -DataTypePtr FunctionNullIf::getReturnTypeImpl(const DataTypes & arguments) const -{ - return FunctionIf{}.getReturnTypeImpl({std::make_shared(), makeNullable(arguments[0]), arguments[0]}); -} - -void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - /// nullIf(col1, col2) == if(col1 == col2, NULL, col1) - - Block temp_block = block; - - size_t res_pos = temp_block.columns(); - temp_block.insert({nullptr, std::make_shared(), ""}); - - { - auto equals_func = FunctionFactory::instance().get("equals", context)->build( - {block.getByPosition(arguments[0]), block.getByPosition(arguments[1])}); - equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count); - } - - /// Argument corresponding to the NULL value. - size_t null_pos = temp_block.columns(); - - /// Append a NULL column. - ColumnWithTypeAndName null_elem; - null_elem.type = block.getByPosition(result).type; - null_elem.column = null_elem.type->createColumnConstWithDefaultValue(input_rows_count); - null_elem.name = "NULL"; - - temp_block.insert(null_elem); - - FunctionIf{}.execute(temp_block, {res_pos, null_pos, arguments[0]}, result, input_rows_count); - - block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); -} - -/// Implementation of assumeNotNull. - -FunctionPtr FunctionAssumeNotNull::create(const Context &) -{ - return std::make_shared(); -} - -std::string FunctionAssumeNotNull::getName() const -{ - return name; -} - -DataTypePtr FunctionAssumeNotNull::getReturnTypeImpl(const DataTypes & arguments) const -{ - return removeNullable(arguments[0]); -} - -void FunctionAssumeNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const ColumnPtr & col = block.getByPosition(arguments[0]).column; - ColumnPtr & res_col = block.getByPosition(result).column; - - if (col->isColumnNullable()) - { - const ColumnNullable & nullable_col = static_cast(*col); - res_col = nullable_col.getNestedColumnPtr(); - } - else - res_col = col; -} - -/// Implementation of toNullable. - -FunctionPtr FunctionToNullable::create(const Context &) -{ - return std::make_shared(); -} - -std::string FunctionToNullable::getName() const -{ - return name; -} - -DataTypePtr FunctionToNullable::getReturnTypeImpl(const DataTypes & arguments) const -{ - return makeNullable(arguments[0]); -} - -void FunctionToNullable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - block.getByPosition(result).column = makeNullable(block.getByPosition(arguments[0]).column); -} - -} diff --git a/dbms/src/Functions/FunctionsNull.h b/dbms/src/Functions/FunctionsNull.h deleted file mode 100644 index 8a0a3048c58..00000000000 --- a/dbms/src/Functions/FunctionsNull.h +++ /dev/null @@ -1,137 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class Block; -class Context; - -/// Implements the function isNull which returns true if a value -/// is null, false otherwise. -class FunctionIsNull : public IFunction -{ -public: - static constexpr auto name = "isNull"; - static FunctionPtr create(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/// Implements the function isNotNull which returns true if a value -/// is not null, false otherwise. -class FunctionIsNotNull : public IFunction -{ -public: - static constexpr auto name = "isNotNull"; - static FunctionPtr create(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/// Implements the function coalesce which takes a set of arguments and -/// returns the value of the leftmost non-null argument. If no such value is -/// found, coalesce() returns NULL. -class FunctionCoalesce : public IFunction -{ -public: - static constexpr auto name = "coalesce"; - static FunctionPtr create(const Context & context); - FunctionCoalesce(const Context & context) : context(context) {} - - std::string getName() const override; - bool useDefaultImplementationForNulls() const override { return false; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - const Context & context; -}; - -/// Implements the function ifNull which takes 2 arguments and returns -/// the value of the 1st argument if it is not null. Otherwise it returns -/// the value of the 2nd argument. -class FunctionIfNull : public IFunction -{ -public: - static constexpr auto name = "ifNull"; - static FunctionPtr create(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/// Implements the function nullIf which takes 2 arguments and returns -/// NULL if both arguments have the same value. Otherwise it returns the -/// value of the first argument. -class FunctionNullIf : public IFunction -{ -private: - const Context & context; -public: - static constexpr auto name = "nullIf"; - static FunctionPtr create(const Context & context); - FunctionNullIf(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/// Implements the function assumeNotNull which takes 1 argument and works as follows: -/// - if the argument is a nullable column, return its embedded column; -/// - otherwise return the original argument. -/// NOTE: assumeNotNull may not be called with the NULL value. -class FunctionAssumeNotNull : public IFunction -{ -public: - static constexpr auto name = "assumeNotNull"; - static FunctionPtr create(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/// If value is not Nullable or NULL, wraps it to Nullable. -class FunctionToNullable : public IFunction -{ -public: - static constexpr auto name = "toNullable"; - static FunctionPtr create(const Context & context); - - std::string getName() const override; - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -} diff --git a/dbms/src/Functions/FunctionsReinterpret.h b/dbms/src/Functions/FunctionsReinterpret.h index 3f90aeabb81..547e565ea6a 100644 --- a/dbms/src/Functions/FunctionsReinterpret.h +++ b/dbms/src/Functions/FunctionsReinterpret.h @@ -167,7 +167,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const IDataType & type = *arguments[0]; - if (!type.isStringOrFixedString()) + if (!isStringOrFixedString(type)) throw Exception("Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 35cfa56426d..db724939977 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -578,7 +578,7 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto & type : arguments) - if (!type->isNumber()) + if (!isNumber(type)) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp deleted file mode 100644 index a7f20a25ae8..00000000000 --- a/dbms/src/Functions/FunctionsString.cpp +++ /dev/null @@ -1,1284 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -#if __SSE2__ -#include -#endif - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int LOGICAL_ERROR; -} - -using namespace GatherUtils; - -template -struct EmptyImpl -{ - /// If the function will return constant value for FixedString data type. - static constexpr auto is_fixed_to_constant = false; - - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - ColumnString::Offset prev_offset = 1; - for (size_t i = 0; i < size; ++i) - { - res[i] = negative ^ (offsets[i] == prev_offset); - prev_offset = offsets[i] + 1; - } - } - - /// Only make sense if is_fixed_to_constant. - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/) - { - throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR); - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) - { - std::vector empty_chars(n); - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n)); - } - - static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - res[i] = negative ^ (offsets[i] == prev_offset); - prev_offset = offsets[i]; - } - } -}; - - -/** Calculates the length of a string in bytes. - */ -struct LengthImpl -{ - static constexpr auto is_fixed_to_constant = true; - - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); - } - - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res) - { - res = n; - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) - { - } - - static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]); - } -}; - - -/** If the string is UTF-8 encoded text, it returns the length of the text in code points. - * (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization) - * (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization) - * Otherwise, the behavior is undefined. - */ -struct LengthUTF8Impl -{ - static constexpr auto is_fixed_to_constant = false; - - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - res[i] = UTF8::countCodePoints(&data[prev_offset], offsets[i] - prev_offset - 1); - prev_offset = offsets[i]; - } - } - - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/) - { - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) - { - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - { - res[i] = UTF8::countCodePoints(&data[i * n], n); - } - } - - static void array(const ColumnString::Offsets &, PaddedPODArray &) - { - throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } -}; - - -template -struct LowerUpperImpl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), res_data.data()); - } - - static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) - { - res_data.resize(data.size()); - array(data.data(), data.data() + data.size(), res_data.data()); - } - -private: - static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) - { - const auto flip_case_mask = 'A' ^ 'a'; - -#if __SSE2__ - const auto bytes_sse = sizeof(__m128i); - const auto src_end_sse = src_end - (src_end - src) % bytes_sse; - - const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); - const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); - const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); - - for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse) - { - /// load 16 sequential 8-bit characters - const auto chars = _mm_loadu_si128(reinterpret_cast(src)); - - /// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound] - const auto is_not_case - = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); - - /// keep `flip_case_mask` only where necessary, zero out elsewhere - const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); - - /// flip case by applying calculated mask - const auto cased_chars = _mm_xor_si128(chars, xor_mask); - - /// store result back to destination - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); - } -#endif - - for (; src < src_end; ++src, ++dst) - if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) - *dst = *src ^ flip_case_mask; - else - *dst = *src; - } -}; - -/** Expands the string in bytes. - */ -struct ReverseImpl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - for (size_t j = prev_offset; j < offsets[i] - 1; ++j) - res_data[j] = data[offsets[i] + prev_offset - 2 - j]; - res_data[offsets[i] - 1] = 0; - prev_offset = offsets[i]; - } - } - - static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data) - { - res_data.resize(data.size()); - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - for (size_t j = i * n; j < (i + 1) * n; ++j) - res_data[j] = data[(i * 2 + 1) * n - j - 1]; - } -}; - - -/** Expands the sequence of code points in a UTF-8 encoded string. - * The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols. - * If the string is not encoded in UTF-8, then the behavior is undefined. - */ -struct ReverseUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - ColumnString::Offset j = prev_offset; - while (j < offsets[i] - 1) - { - if (data[j] < 0xBF) - { - res_data[offsets[i] + prev_offset - 2 - j] = data[j]; - j += 1; - } - else if (data[j] < 0xE0) - { - memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 1], &data[j], 2); - j += 2; - } - else if (data[j] < 0xF0) - { - memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 2], &data[j], 3); - j += 3; - } - else - { - res_data[offsets[i] + prev_offset - 2 - j] = data[j]; - j += 1; - } - } - - res_data[offsets[i] - 1] = 0; - prev_offset = offsets[i]; - } - } - - static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) - { - throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -template -void LowerUpperUTF8Impl::vector(const ColumnString::Chars_t & data, - const IColumn::Offsets & offsets, - ColumnString::Chars_t & res_data, - IColumn::Offsets & res_offsets) -{ - res_data.resize(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), res_data.data()); -} - -template -void LowerUpperUTF8Impl::vector_fixed( - const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) -{ - res_data.resize(data.size()); - array(data.data(), data.data() + data.size(), res_data.data()); -} - -template -void LowerUpperUTF8Impl::constant( - const std::string & data, std::string & res_data) -{ - res_data.resize(data.size()); - array(reinterpret_cast(data.data()), - reinterpret_cast(data.data() + data.size()), - reinterpret_cast(res_data.data())); -} - -template -void LowerUpperUTF8Impl::toCase( - const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) -{ - if (src[0] <= ascii_upper_bound) - { - if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) - *dst++ = *src++ ^ flip_case_mask; - else - *dst++ = *src++; - } - else if (src + 1 < src_end - && ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu)))) - { - cyrillic_to_case(src, dst); - } - else if (src + 1 < src_end && src[0] == 0xC2u) - { - /// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF - *dst++ = *src++; - *dst++ = *src++; - } - else if (src + 2 < src_end && src[0] == 0xE2u) - { - /// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF - *dst++ = *src++; - *dst++ = *src++; - *dst++ = *src++; - } - else - { - static const Poco::UTF8Encoding utf8; - - if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src)) - { - src += chars; - dst += chars; - } - else - { - ++src; - ++dst; - } - } -} - -template -void LowerUpperUTF8Impl::array( - const UInt8 * src, const UInt8 * src_end, UInt8 * dst) -{ -#if __SSE2__ - const auto bytes_sse = sizeof(__m128i); - auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; - - /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) - const auto v_zero = _mm_setzero_si128(); - const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); - const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); - const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); - - while (src < src_end_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(src)); - - /// check for ASCII - const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero); - const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii); - - /// ASCII - if (mask_is_not_ascii == 0) - { - const auto is_not_case - = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); - const auto mask_is_not_case = _mm_movemask_epi8(is_not_case); - - /// everything in correct case ASCII - if (mask_is_not_case == 0) - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars); - else - { - /// ASCII in mixed case - /// keep `flip_case_mask` only where necessary, zero out elsewhere - const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); - - /// flip case by applying calculated mask - const auto cased_chars = _mm_xor_si128(chars, xor_mask); - - /// store result back to destination - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); - } - - src += bytes_sse; - dst += bytes_sse; - } - else - { - /// UTF-8 - const auto expected_end = src + bytes_sse; - - while (src < expected_end) - toCase(src, src_end, dst); - - /// adjust src_end_sse by pushing it forward or backward - const auto diff = src - expected_end; - if (diff != 0) - { - if (src_end_sse + diff < src_end) - src_end_sse += diff; - else - src_end_sse -= bytes_sse - diff; - } - } - } -#endif - /// handle remaining symbols - while (src < src_end) - toCase(src, src_end, dst); -} - - -/** If the string is encoded in UTF-8, then it selects a substring of code points in it. - * Otherwise, the behavior is undefined. - */ -struct SubstringUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - size_t start, - size_t length, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.reserve(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); - - ColumnString::Offset prev_offset = 0; - ColumnString::Offset res_offset = 0; - for (size_t i = 0; i < size; ++i) - { - ColumnString::Offset j = prev_offset; - ColumnString::Offset pos = 1; - ColumnString::Offset bytes_start = 0; - ColumnString::Offset bytes_length = 0; - while (j < offsets[i] - 1) - { - if (pos == start) - bytes_start = j - prev_offset + 1; - - if (data[j] < 0xBF) - j += 1; - else if (data[j] < 0xE0) - j += 2; - else if (data[j] < 0xF0) - j += 3; - else - j += 1; - - if (pos >= start && pos < start + length) - bytes_length = j - prev_offset + 1 - bytes_start; - else if (pos >= start + length) - break; - - ++pos; - } - - if (bytes_start == 0) - { - res_data.resize(res_data.size() + 1); - res_data[res_offset] = 0; - ++res_offset; - } - else - { - size_t bytes_to_copy = std::min(offsets[i] - prev_offset - bytes_start, bytes_length); - res_data.resize(res_data.size() + bytes_to_copy + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + bytes_start - 1], bytes_to_copy); - res_offset += bytes_to_copy + 1; - res_data[res_offset - 1] = 0; - } - res_offsets[i] = res_offset; - prev_offset = offsets[i]; - } - } -}; - - -template -class FunctionStringOrArrayToT : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isStringOrFixedString() - && !checkDataType(arguments[0].get())) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared>(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::vector(col->getChars(), col->getOffsets(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - if (Impl::is_fixed_to_constant) - { - ResultType res = 0; - Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res); - - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res)); - } - else - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - } - else if (const ColumnArray * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::array(col->getOffsets(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -/// Also works with arrays. -class FunctionReverse : public IFunction -{ -public: - static constexpr auto name = "reverse"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isInjective(const Block &) override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isStringOrFixedString() - && !checkDataType(arguments[0].get())) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnFixedString::create(col->getN()); - ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); - block.getByPosition(result).column = std::move(col_res); - } - else if (checkColumn(column.get())) - { - FunctionArrayReverse().execute(block, arguments, result, input_rows_count); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -template -class ConcatImpl : public IFunction -{ -public: - static constexpr auto name = Name::name; - ConcatImpl(const Context & context) : context(context) {} - static FunctionPtr create(const Context & context) - { - return std::make_shared(context); - } - - String getName() const override - { - return name; - } - - bool isVariadic() const override - { - return true; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isInjective(const Block &) override - { - return is_injective; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!is_injective && !arguments.empty() && checkDataType(arguments[0].get())) - return FunctionArrayConcat(context).getReturnTypeImpl(arguments); - - if (arguments.size() < 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) - + ", should be at least 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - for (const auto arg_idx : ext::range(0, arguments.size())) - { - const auto arg = arguments[arg_idx].get(); - if (!arg->isStringOrFixedString()) - throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - if (!is_injective && !arguments.empty() && checkDataType(block.getByPosition(arguments[0]).type.get())) - return FunctionArrayConcat(context).executeImpl(block, arguments, result, input_rows_count); - - if (arguments.size() == 2) - executeBinary(block, arguments, result, input_rows_count); - else - executeNAry(block, arguments, result, input_rows_count); - } - -private: - const Context & context; - - void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) - { - const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); - const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); - - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); - const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); - const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); - - auto c_res = ColumnString::create(); - - if (c0_string && c1_string) - concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); - else if (c0_string && c1_const_string) - concat(StringSource(*c0_string), ConstSource(*c1_const_string), StringSink(*c_res, c0->size())); - else if (c0_const_string && c1_string) - concat(ConstSource(*c0_const_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); - else - { - /// Fallback: use generic implementation for not very important cases. - executeNAry(block, arguments, result, input_rows_count); - return; - } - - block.getByPosition(result).column = std::move(c_res); - } - - void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) - { - size_t num_sources = arguments.size(); - StringSources sources(num_sources); - - for (size_t i = 0; i < num_sources; ++i) - sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column); - - auto c_res = ColumnString::create(); - concat(sources, StringSink(*c_res, input_rows_count)); - block.getByPosition(result).column = std::move(c_res); - } -}; - - -class FunctionSubstring : public IFunction -{ -public: - static constexpr auto name = "substring"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - 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 - { - size_t number_of_arguments = arguments.size(); - - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(number_of_arguments) + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!arguments[0]->isStringOrFixedString()) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!arguments[1]->isNumber()) - throw Exception("Illegal type " + arguments[1]->getName() - + " of second argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (number_of_arguments == 3 && !arguments[2]->isNumber()) - throw Exception("Illegal type " + arguments[2]->getName() - + " of second argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - template - void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length, - const ColumnConst * column_start_const, const ColumnConst * column_length_const, - Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source, - size_t input_rows_count) - { - auto col_res = ColumnString::create(); - - if (!column_length) - { - if (column_start_const) - { - if (start_value > 0) - sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1); - else if (start_value < 0) - sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value); - else - throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - } - else - sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); - } - else - { - if (column_start_const && column_length_const) - { - if (start_value > 0) - sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value); - else if (start_value < 0) - sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value); - else - throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - } - else - sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length); - } - - block.getByPosition(result).column = std::move(col_res); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - size_t number_of_arguments = arguments.size(); - - ColumnPtr column_string = block.getByPosition(arguments[0]).column; - ColumnPtr column_start = block.getByPosition(arguments[1]).column; - ColumnPtr column_length; - - if (number_of_arguments == 3) - column_length = block.getByPosition(arguments[2]).column; - - const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); - const ColumnConst * column_length_const = nullptr; - - if (number_of_arguments == 3) - column_length_const = checkAndGetColumn(column_length.get()); - - Int64 start_value = 0; - Int64 length_value = 0; - - if (column_start_const) - { - start_value = column_start_const->getInt(0); - } - if (column_length_const) - { - length_value = column_length_const->getInt(0); - if (length_value < 0) - throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } - - if (const ColumnString * col = checkAndGetColumn(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, StringSource(*col), input_rows_count); - else if (const ColumnFixedString * col = checkAndGetColumn(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, FixedStringSource(*col), input_rows_count); - else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, ConstSource(*col), input_rows_count); - else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, ConstSource(*col), input_rows_count); - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -class FunctionSubstringUTF8 : public IFunction -{ -public: - static constexpr auto name = "substringUTF8"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 3; - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isString()) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!arguments[1]->isNumber() || !arguments[2]->isNumber()) - throw Exception("Illegal type " + (arguments[1]->isNumber() ? arguments[2]->getName() : arguments[1]->getName()) - + " of argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column_string = block.getByPosition(arguments[0]).column; - const ColumnPtr column_start = block.getByPosition(arguments[1]).column; - const ColumnPtr column_length = block.getByPosition(arguments[2]).column; - - if (!column_start->isColumnConst() || !column_length->isColumnConst()) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); - - Field start_field = (*block.getByPosition(arguments[1]).column)[0]; - Field length_field = (*block.getByPosition(arguments[2]).column)[0]; - - if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type."); - - UInt64 start = start_field.get(); - UInt64 length = length_field.get(); - - if (start == 0) - throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - /// Otherwise may lead to overflow and pass bounds check inside inner loop. - if (start >= 0x8000000000000000ULL || length >= 0x8000000000000000ULL) - throw Exception("Too large values of 2nd or 3rd argument provided for function substring.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (const ColumnString * col = checkAndGetColumn(column_string.get())) - { - auto col_res = ColumnString::create(); - SubstringUTF8Impl::vector(col->getChars(), col->getOffsets(), start, length, col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -class FunctionAppendTrailingCharIfAbsent : public IFunction -{ -public: - static constexpr auto name = "appendTrailingCharIfAbsent"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - -private: - size_t getNumberOfArguments() const override - { - return 2; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isString()) - throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - if (!arguments[1]->isString()) - throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto & column = block.getByPosition(arguments[0]).column; - const auto & column_char = block.getByPosition(arguments[1]).column; - - if (!checkColumnConst(column_char.get())) - throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; - - String trailing_char_str = static_cast(*column_char).getValue(); - - if (trailing_char_str.size() != 1) - throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS}; - - if (const auto col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - - const auto & src_data = col->getChars(); - const auto & src_offsets = col->getOffsets(); - - auto & dst_data = col_res->getChars(); - auto & dst_offsets = col_res->getOffsets(); - - const auto size = src_offsets.size(); - dst_data.resize(src_data.size() + size); - dst_offsets.resize(size); - - ColumnString::Offset src_offset{}; - ColumnString::Offset dst_offset{}; - - for (const auto i : ext::range(0, size)) - { - const auto src_length = src_offsets[i] - src_offset; - memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length); - src_offset = src_offsets[i]; - dst_offset += src_length; - - if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) - { - dst_data[dst_offset - 1] = trailing_char_str.front(); - dst_data[dst_offset] = 0; - ++dst_offset; - } - - dst_offsets[i] = dst_offset; - } - - dst_data.resize_assume_reserved(dst_offset); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN}; - } -}; - - -struct NameStartsWith -{ - static constexpr auto name = "startsWith"; -}; -struct NameEndsWith -{ - static constexpr auto name = "endsWith"; -}; - -template -class FunctionStartsEndsWith : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isStringOrFixedString()) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!arguments[1]->isStringOrFixedString()) - throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared>(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); - const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); - - auto col_res = ColumnVector::create(); - typename ColumnVector::Container & vec_res = col_res->getData(); - - vec_res.resize(input_rows_count); - - if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) - dispatch(StringSource(*haystack), needle_column, vec_res); - else if (const ColumnFixedString * haystack = checkAndGetColumn(haystack_column)) - dispatch(FixedStringSource(*haystack), needle_column, vec_res); - else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) - dispatch>(ConstSource(*haystack), needle_column, vec_res); - else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) - dispatch>(ConstSource(*haystack), needle_column, vec_res); - else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - - block.getByPosition(result).column = std::move(col_res); - } - -private: - template - void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const - { - if (const ColumnString * needle = checkAndGetColumn(needle_column)) - execute(haystack_source, StringSource(*needle), res_data); - else if (const ColumnFixedString * needle = checkAndGetColumn(needle_column)) - execute(haystack_source, FixedStringSource(*needle), res_data); - else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) - execute>(haystack_source, ConstSource(*needle), res_data); - else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) - execute>(haystack_source, ConstSource(*needle), res_data); - else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - } - - template - static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) - { - size_t row_num = 0; - - while (!haystack_source.isEnd()) - { - auto haystack = haystack_source.getWhole(); - auto needle = needle_source.getWhole(); - - if (needle.size > haystack.size) - { - res_data[row_num] = false; - } - else - { - if constexpr (std::is_same_v) - { - res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); - } - else /// endsWith - { - res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); - } - } - - haystack_source.next(); - needle_source.next(); - ++row_num; - } - } -}; - - -struct NameEmpty -{ - static constexpr auto name = "empty"; -}; -struct NameNotEmpty -{ - static constexpr auto name = "notEmpty"; -}; -struct NameLength -{ - static constexpr auto name = "length"; -}; -struct NameLengthUTF8 -{ - static constexpr auto name = "lengthUTF8"; -}; -struct NameLower -{ - static constexpr auto name = "lower"; -}; -struct NameUpper -{ - static constexpr auto name = "upper"; -}; -struct NameReverseUTF8 -{ - static constexpr auto name = "reverseUTF8"; -}; -struct NameConcat -{ - static constexpr auto name = "concat"; -}; -struct NameConcatAssumeInjective -{ - static constexpr auto name = "concatAssumeInjective"; -}; - - -using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8>; -using FunctionNotEmpty = FunctionStringOrArrayToT, NameNotEmpty, UInt8>; -using FunctionLength = FunctionStringOrArrayToT; -using FunctionLengthUTF8 = FunctionStringOrArrayToT; -using FunctionLower = FunctionStringToString, NameLower>; -using FunctionUpper = FunctionStringToString, NameUpper>; -using FunctionReverseUTF8 = FunctionStringToString; -using FunctionConcat = ConcatImpl; -using FunctionConcatAssumeInjective = ConcatImpl; -using FunctionStartsWith = FunctionStartsEndsWith; -using FunctionEndsWith = FunctionStartsEndsWith; - - -void registerFunctionsString(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} -} diff --git a/dbms/src/Functions/FunctionsString.h b/dbms/src/Functions/FunctionsString.h deleted file mode 100644 index 428c9df679d..00000000000 --- a/dbms/src/Functions/FunctionsString.h +++ /dev/null @@ -1,204 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} - -/** String functions - * - * length, empty, notEmpty, - * concat, substring, lower, upper, reverse - * lengthUTF8, substringUTF8, lowerUTF8, upperUTF8, reverseUTF8 - * - * s -> UInt8: empty, notEmpty - * s -> UInt64: length, lengthUTF8 - * s -> s: lower, upper, lowerUTF8, upperUTF8, reverse, reverseUTF8 - * s, s -> s: concat - * s, c1, c2 -> s: substring, substringUTF8 - * s, c1, c2, s2 -> s: replace, replaceUTF8 - * - * The search functions for strings and regular expressions are located separately. - * URL functions are located separately. - * String encoding functions, converting to other types are located separately. - * - * The functions length, empty, notEmpty, reverse also work with arrays. - */ - - -/// xor or do nothing -template -UInt8 xor_or_identity(const UInt8 c, const int mask) -{ - return c ^ mask; -} - -template <> -inline UInt8 xor_or_identity(const UInt8 c, const int) -{ - return c; -} - -/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array -template -inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) -{ - if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// ѐёђѓєѕіїјљњћќѝўџ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// А-П - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) - { - /// а-п - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) - { - /// Р-Я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// р-я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } -} - - -/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case. - * Note: It is assumed that after the character is converted to another case, - * the length of its multibyte sequence in UTF-8 does not change. - * Otherwise, the behavior is undefined. - */ -template -struct LowerUpperUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets); - - static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data); - - static void constant(const std::string & data, std::string & res_data); - - /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. - * `src` and `dst` are incremented by corresponding sequence lengths. */ - static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst); - -private: - static constexpr auto ascii_upper_bound = '\x7f'; - static constexpr auto flip_case_mask = 'A' ^ 'a'; - - static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst); -}; - - -template -class FunctionStringToString : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isInjective(const Block &) override - { - return is_injective; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isStringOrFixedString()) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnFixedString::create(col->getN()); - Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -struct NameLowerUTF8 -{ - static constexpr auto name = "lowerUTF8"; -}; -struct NameUpperUTF8 -{ - static constexpr auto name = "upperUTF8"; -}; - - -using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; -using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; - -} diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index 2e06305ef1f..615e9cdb7cc 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -66,7 +66,7 @@ public: /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -124,11 +124,11 @@ public: static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -504,11 +504,11 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type || !array_type->getNestedType()->isString()) + if (!array_type || !isString(array_type->getNestedType())) throw Exception("First argument for function " + getName() + " must be array of strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2 - && !arguments[1]->isString()) + && !isString(arguments[1])) throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 2c8b575b6b8..243120d767f 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -954,15 +954,15 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[1]->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[1])) throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[2]->isStringOrFixedString()) + if (!isStringOrFixedString(arguments[2])) throw Exception("Illegal type " + arguments[2]->getName() + " of third argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index 462631c28f4..e679a811027 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -65,11 +65,11 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception( "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception( "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -149,11 +149,11 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception( "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!arguments[1]->isString()) + if (!isString(arguments[1])) throw Exception( "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Functions/FunctionsTransform.h b/dbms/src/Functions/FunctionsTransform.h index 0e8eea67779..b3d14a927f4 100644 --- a/dbms/src/Functions/FunctionsTransform.h +++ b/dbms/src/Functions/FunctionsTransform.h @@ -77,7 +77,7 @@ public: const DataTypePtr & type_x = arguments[0]; - if (!type_x->isValueRepresentedByNumber() && !type_x->isString()) + if (!type_x->isValueRepresentedByNumber() && !isString(type_x)) throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -91,7 +91,7 @@ public: const auto type_arr_from_nested = type_arr_from->getNestedType(); if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber()) - || (!!type_x->isString() != !!type_arr_from_nested->isString())) + || (isString(type_x) != isString(type_arr_from_nested))) { throw Exception{"First argument and elements of array of second argument of function " + getName() + " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -108,7 +108,7 @@ public: if (args_size == 3) { if ((type_x->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber()) - || (!!type_x->isString() != !!checkDataType(type_arr_to_nested.get()))) + || (isString(type_x) != isString(type_arr_to_nested))) throw Exception{"Function " + getName() + " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -119,13 +119,16 @@ public: { const DataTypePtr & type_default = arguments[3]; - if (!type_default->isValueRepresentedByNumber() && !type_default->isString()) + if (!type_default->isValueRepresentedByNumber() && !isString(type_default)) throw Exception{"Unsupported type " + type_default->getName() + " of fourth argument (default value) of function " + getName() + ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + bool default_is_string = WhichDataType(type_default).isString(); + bool nested_is_string = WhichDataType(type_arr_to_nested).isString(); + if ((type_default->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber()) - || (!!checkDataType(type_default.get()) != !!checkDataType(type_arr_to_nested.get()))) + || (default_is_string != nested_is_string)) throw Exception{"Function " + getName() + " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index 4d449d4f27a..f2bcf266db2 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include @@ -584,7 +584,7 @@ public: static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -669,7 +669,7 @@ public: static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -746,7 +746,7 @@ public: static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -841,7 +841,7 @@ public: static void checkArguments(const DataTypes & arguments) { - if (!arguments[0]->isString()) + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } diff --git a/dbms/src/Functions/FunctionsVisitParam.cpp b/dbms/src/Functions/FunctionsVisitParam.cpp index 0870d96d094..650b569bc2a 100644 --- a/dbms/src/Functions/FunctionsVisitParam.cpp +++ b/dbms/src/Functions/FunctionsVisitParam.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include @@ -26,7 +25,6 @@ using FunctionVisitParamExtractRaw = FunctionsStringSearchToString, NameVisitParamExtractString>; - void registerFunctionsVisitParam(FunctionFactory & factory) { factory.registerFunction(); diff --git a/dbms/src/Functions/GatherUtils/GatherUtils.h b/dbms/src/Functions/GatherUtils/GatherUtils.h index da143af6fcf..3b73994592d 100644 --- a/dbms/src/Functions/GatherUtils/GatherUtils.h +++ b/dbms/src/Functions/GatherUtils/GatherUtils.h @@ -37,15 +37,12 @@ std::unique_ptr createArraySink(ColumnArray & col, size_t column_siz void concat(const std::vector> & sources, IArraySink & sink); void sliceFromLeftConstantOffsetUnbounded(IArraySource & src, IArraySink & sink, size_t offset); - void sliceFromLeftConstantOffsetBounded(IArraySource & src, IArraySink & sink, size_t offset, ssize_t length); void sliceFromRightConstantOffsetUnbounded(IArraySource & src, IArraySink & sink, size_t offset); - void sliceFromRightConstantOffsetBounded(IArraySource & src, IArraySink & sink, size_t offset, ssize_t length); void sliceDynamicOffsetUnbounded(IArraySource & src, IArraySink & sink, const IColumn & offset_column); - void sliceDynamicOffsetBounded(IArraySource & src, IArraySink & sink, const IColumn & offset_column, const IColumn & length_column); void sliceHas(IArraySource & first, IArraySource & second, bool all, ColumnUInt8 & result); @@ -55,5 +52,6 @@ void push(IArraySource & array_source, IValueSource & value_source, IArraySink & void resizeDynamicSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, const IColumn & size_column); void resizeConstantSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, ssize_t size); + } diff --git a/dbms/src/Functions/LowerUpperImpl.h b/dbms/src/Functions/LowerUpperImpl.h new file mode 100644 index 00000000000..50806372188 --- /dev/null +++ b/dbms/src/Functions/LowerUpperImpl.h @@ -0,0 +1,67 @@ +#include + + +namespace DB +{ + +template +struct LowerUpperImpl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) + { + res_data.resize(data.size()); + array(data.data(), data.data() + data.size(), res_data.data()); + } + +private: + static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) + { + const auto flip_case_mask = 'A' ^ 'a'; + +#if __SSE2__ + const auto bytes_sse = sizeof(__m128i); + const auto src_end_sse = src_end - (src_end - src) % bytes_sse; + + const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); + const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); + const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); + + for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse) + { + /// load 16 sequential 8-bit characters + const auto chars = _mm_loadu_si128(reinterpret_cast(src)); + + /// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound] + const auto is_not_case + = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); + + /// keep `flip_case_mask` only where necessary, zero out elsewhere + const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); + + /// flip case by applying calculated mask + const auto cased_chars = _mm_xor_si128(chars, xor_mask); + + /// store result back to destination + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); + } +#endif + + for (; src < src_end; ++src, ++dst) + if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) + *dst = *src ^ flip_case_mask; + else + *dst = *src; + } +}; + +} diff --git a/dbms/src/Functions/LowerUpperUTF8Impl.h b/dbms/src/Functions/LowerUpperUTF8Impl.h new file mode 100644 index 00000000000..8e67a7976b2 --- /dev/null +++ b/dbms/src/Functions/LowerUpperUTF8Impl.h @@ -0,0 +1,229 @@ +#include +#include + +#if __SSE2__ +#include +#endif + + +namespace DB +{ + +namespace +{ + /// xor or do nothing + template + UInt8 xor_or_identity(const UInt8 c, const int mask) + { + return c ^ mask; + } + + template <> + inline UInt8 xor_or_identity(const UInt8 c, const int) + { + return c; + } + + /// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array + template + inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) + { + if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// ѐёђѓєѕіїјљњћќѝўџ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// А-П + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) + { + /// а-п + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) + { + /// Р-Я + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// р-я + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x20); + } + } +} + + +/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case. + * Note: It is assumed that after the character is converted to another case, + * the length of its multibyte sequence in UTF-8 does not change. + * Otherwise, the behavior is undefined. + */ +template +struct LowerUpperUTF8Impl +{ + static void vector( + const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) + { + res_data.resize(data.size()); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void constant(const std::string & data, std::string & res_data) + { + res_data.resize(data.size()); + array(reinterpret_cast(data.data()), + reinterpret_cast(data.data() + data.size()), + reinterpret_cast(res_data.data())); + } + + /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. + * `src` and `dst` are incremented by corresponding sequence lengths. */ + static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) + { + if (src[0] <= ascii_upper_bound) + { + if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) + *dst++ = *src++ ^ flip_case_mask; + else + *dst++ = *src++; + } + else if (src + 1 < src_end + && ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu)))) + { + cyrillic_to_case(src, dst); + } + else if (src + 1 < src_end && src[0] == 0xC2u) + { + /// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF + *dst++ = *src++; + *dst++ = *src++; + } + else if (src + 2 < src_end && src[0] == 0xE2u) + { + /// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF + *dst++ = *src++; + *dst++ = *src++; + *dst++ = *src++; + } + else + { + static const Poco::UTF8Encoding utf8; + + if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src)) + { + src += chars; + dst += chars; + } + else + { + ++src; + ++dst; + } + } + } + +private: + static constexpr auto ascii_upper_bound = '\x7f'; + static constexpr auto flip_case_mask = 'A' ^ 'a'; + + static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) + { +#if __SSE2__ + const auto bytes_sse = sizeof(__m128i); + auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; + + /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) + const auto v_zero = _mm_setzero_si128(); + const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); + const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); + const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); + + while (src < src_end_sse) + { + const auto chars = _mm_loadu_si128(reinterpret_cast(src)); + + /// check for ASCII + const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero); + const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii); + + /// ASCII + if (mask_is_not_ascii == 0) + { + const auto is_not_case + = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); + const auto mask_is_not_case = _mm_movemask_epi8(is_not_case); + + /// everything in correct case ASCII + if (mask_is_not_case == 0) + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars); + else + { + /// ASCII in mixed case + /// keep `flip_case_mask` only where necessary, zero out elsewhere + const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); + + /// flip case by applying calculated mask + const auto cased_chars = _mm_xor_si128(chars, xor_mask); + + /// store result back to destination + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); + } + + src += bytes_sse; + dst += bytes_sse; + } + else + { + /// UTF-8 + const auto expected_end = src + bytes_sse; + + while (src < expected_end) + toCase(src, src_end, dst); + + /// adjust src_end_sse by pushing it forward or backward + const auto diff = src - expected_end; + if (diff != 0) + { + if (src_end_sse + diff < src_end) + src_end_sse += diff; + else + src_end_sse -= bytes_sse - diff; + } + } + } +#endif + /// handle remaining symbols + while (src < src_end) + toCase(src, src_end, dst); + } +}; + +} diff --git a/dbms/src/Functions/appendTrailingCharIfAbsent.cpp b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp new file mode 100644 index 00000000000..fe28a6026bd --- /dev/null +++ b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + + +class FunctionAppendTrailingCharIfAbsent : public IFunction +{ +public: + static constexpr auto name = "appendTrailingCharIfAbsent"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + +private: + size_t getNumberOfArguments() const override + { + return 2; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!isString(arguments[1])) + throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto & column = block.getByPosition(arguments[0]).column; + const auto & column_char = block.getByPosition(arguments[1]).column; + + if (!checkColumnConst(column_char.get())) + throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; + + String trailing_char_str = static_cast(*column_char).getValue(); + + if (trailing_char_str.size() != 1) + throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS}; + + if (const auto col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + + const auto & src_data = col->getChars(); + const auto & src_offsets = col->getOffsets(); + + auto & dst_data = col_res->getChars(); + auto & dst_offsets = col_res->getOffsets(); + + const auto size = src_offsets.size(); + dst_data.resize(src_data.size() + size); + dst_offsets.resize(size); + + ColumnString::Offset src_offset{}; + ColumnString::Offset dst_offset{}; + + for (const auto i : ext::range(0, size)) + { + const auto src_length = src_offsets[i] - src_offset; + memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length); + src_offset = src_offsets[i]; + dst_offset += src_length; + + if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) + { + dst_data[dst_offset - 1] = trailing_char_str.front(); + dst_data[dst_offset] = 0; + ++dst_offset; + } + + dst_offsets[i] = dst_offset; + } + + dst_data.resize_assume_reserved(dst_offset); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN}; + } +}; + +void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/array.cpp b/dbms/src/Functions/array.cpp new file mode 100644 index 00000000000..c0420b83db9 --- /dev/null +++ b/dbms/src/Functions/array.cpp @@ -0,0 +1,119 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// array(c1, c2, ...) - create an array. +class FunctionArray : public IFunction +{ +public: + static constexpr auto name = "array"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + FunctionArray(const Context & context) + : context(context) + { + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return std::make_shared(getLeastSupertype(arguments)); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t num_elements = arguments.size(); + + if (num_elements == 0) + { + /// We should return constant empty array. + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + const DataTypePtr & return_type = block.getByPosition(result).type; + const DataTypePtr & elem_type = static_cast(*return_type).getNestedType(); + + size_t block_size = input_rows_count; + + /** If part of columns have not same type as common type of all elements of array, + * then convert them to common type. + * If part of columns are constants, + * then convert them to full columns. + */ + + Columns columns_holder(num_elements); + const IColumn * columns[num_elements]; + + for (size_t i = 0; i < num_elements; ++i) + { + const auto & arg = block.getByPosition(arguments[i]); + + ColumnPtr preprocessed_column = arg.column; + + if (!arg.type->equals(*elem_type)) + preprocessed_column = castColumn(arg, elem_type, context); + + if (ColumnPtr materialized_column = preprocessed_column->convertToFullColumnIfConst()) + preprocessed_column = materialized_column; + + columns_holder[i] = std::move(preprocessed_column); + columns[i] = columns_holder[i].get(); + } + + /// Create and fill the result array. + + auto out = ColumnArray::create(elem_type->createColumn()); + IColumn & out_data = out->getData(); + IColumn::Offsets & out_offsets = out->getOffsets(); + + out_data.reserve(block_size * num_elements); + out_offsets.resize(block_size); + + IColumn::Offset current_offset = 0; + for (size_t i = 0; i < block_size; ++i) + { + for (size_t j = 0; j < num_elements; ++j) + out_data.insertFrom(*columns[j], i); + + current_offset += num_elements; + out_offsets[i] = current_offset; + } + + block.getByPosition(result).column = std::move(out); + } + +private: + String getName() const override + { + return name; + } + + bool addField(DataTypePtr type_res, const Field & f, Array & arr) const; + +private: + const Context & context; +}; + + +void registerFunctionArray(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayConcat.cpp b/dbms/src/Functions/arrayConcat.cpp new file mode 100644 index 00000000000..42b92116bf2 --- /dev/null +++ b/dbms/src/Functions/arrayConcat.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// arrayConcat(arr1, ...) - concatenate arrays. +class FunctionArrayConcat : public IFunction +{ +public: + static constexpr auto name = "arrayConcat"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayConcat(const Context & context) : context(context) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + for (auto i : ext::range(0, arguments.size())) + { + auto array_type = typeid_cast(arguments[i].get()); + if (!array_type) + throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " + + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return getLeastSupertype(arguments); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const DataTypePtr & return_type = block.getByPosition(result).type; + + if (return_type->onlyNull()) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto result_column = return_type->createColumn(); + + size_t rows = input_rows_count; + size_t num_args = arguments.size(); + + Columns preprocessed_columns(num_args); + + for (size_t i = 0; i < num_args; ++i) + { + const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]); + ColumnPtr preprocessed_column = arg.column; + + if (!arg.type->equals(*return_type)) + preprocessed_column = castColumn(arg, return_type, context); + + preprocessed_columns[i] = std::move(preprocessed_column); + } + + std::vector> sources; + + for (auto & argument_column : preprocessed_columns) + { + bool is_const = false; + + if (auto argument_column_const = typeid_cast(argument_column.get())) + { + is_const = true; + argument_column = argument_column_const->getDataColumnPtr(); + } + + if (auto argument_column_array = typeid_cast(argument_column.get())) + sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows)); + else + throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; + } + + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), rows); + GatherUtils::concat(sources, *sink); + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + +private: + const Context & context; +}; + + +void registerFunctionArrayConcat(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayDistinct.cpp b/dbms/src/Functions/arrayDistinct.cpp new file mode 100644 index 00000000000..4a0f6fe1d6b --- /dev/null +++ b/dbms/src/Functions/arrayDistinct.cpp @@ -0,0 +1,302 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// Find different elements in an array. +class FunctionArrayDistinct : public IFunction +{ +public: + static constexpr auto name = "arrayDistinct"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return false; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception("Argument for function " + getName() + " must be array but it " + " has type " + arguments[0]->getName() + ".", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto nested_type = removeNullable(array_type->getNestedType()); + + return std::make_shared(nested_type); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. + static constexpr size_t INITIAL_SIZE_DEGREE = 9; + + template + bool executeNumber( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col); + + bool executeString( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col); + + void executeHashed( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col); +}; + + +void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) +{ + ColumnPtr array_ptr = block.getByPosition(arguments[0]).column; + const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + + const auto & return_type = block.getByPosition(result).type; + + auto res_ptr = return_type->createColumn(); + ColumnArray & res = static_cast(*res_ptr); + + const IColumn & src_data = array->getData(); + const ColumnArray::Offsets & offsets = array->getOffsets(); + + IColumn & res_data = res.getData(); + ColumnArray::Offsets & res_offsets = res.getOffsets(); + + const ColumnNullable * nullable_col = nullptr; + + const IColumn * inner_col; + + if (src_data.isColumnNullable()) + { + nullable_col = static_cast(&src_data); + inner_col = &nullable_col->getNestedColumn(); + } + else + { + inner_col = &src_data; + } + + if (!(executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeNumber(*inner_col, offsets, res_data, res_offsets, nullable_col) + || executeString(*inner_col, offsets, res_data, res_offsets, nullable_col))) + executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col); + + block.getByPosition(result).column = std::move(res_ptr); +} + +template +bool FunctionArrayDistinct::executeNumber( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col) +{ + const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data); + + if (!src_data_concrete) + { + return false; + } + + const PaddedPODArray & values = src_data_concrete->getData(); + PaddedPODArray & res_data = typeid_cast &>(res_data_col).getData(); + + const PaddedPODArray * src_null_map = nullptr; + + if (nullable_col) + src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + + using Set = ClearableHashSet, + HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + + Set set; + + ColumnArray::Offset prev_src_offset = 0; + ColumnArray::Offset res_offset = 0; + + for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) + { + set.clear(); + + ColumnArray::Offset curr_src_offset = src_offsets[i]; + for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) + { + if (nullable_col && (*src_null_map)[j]) + continue; + + if (set.find(values[j]) == set.end()) + { + res_data.emplace_back(values[j]); + set.insert(values[j]); + } + } + + res_offset += set.size(); + res_offsets.emplace_back(res_offset); + + prev_src_offset = curr_src_offset; + } + return true; +} + +bool FunctionArrayDistinct::executeString( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col) +{ + const ColumnString * src_data_concrete = checkAndGetColumn(&src_data); + + if (!src_data_concrete) + return false; + + ColumnString & res_data_column_string = typeid_cast(res_data_col); + + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + + const PaddedPODArray * src_null_map = nullptr; + + if (nullable_col) + src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + + Set set; + + ColumnArray::Offset prev_src_offset = 0; + ColumnArray::Offset res_offset = 0; + + for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) + { + set.clear(); + + ColumnArray::Offset curr_src_offset = src_offsets[i]; + for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) + { + if (nullable_col && (*src_null_map)[j]) + continue; + + StringRef str_ref = src_data_concrete->getDataAt(j); + + if (set.find(str_ref) == set.end()) + { + set.insert(str_ref); + res_data_column_string.insertData(str_ref.data, str_ref.size); + } + } + + res_offset += set.size(); + res_offsets.emplace_back(res_offset); + + prev_src_offset = curr_src_offset; + } + return true; +} + +void FunctionArrayDistinct::executeHashed( + const IColumn & src_data, + const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + ColumnArray::Offsets & res_offsets, + const ColumnNullable * nullable_col) +{ + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + + const PaddedPODArray * src_null_map = nullptr; + + if (nullable_col) + src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + + Set set; + + ColumnArray::Offset prev_src_offset = 0; + ColumnArray::Offset res_offset = 0; + + for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i) + { + set.clear(); + + ColumnArray::Offset curr_src_offset = src_offsets[i]; + for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) + { + if (nullable_col && (*src_null_map)[j]) + continue; + + UInt128 hash; + SipHash hash_function; + src_data.updateHashWithValue(j, hash_function); + hash_function.get128(reinterpret_cast(&hash)); + + if (set.find(hash) == set.end()) + { + set.insert(hash); + res_data_col.insertFrom(src_data, j); + } + } + + res_offset += set.size(); + res_offsets.emplace_back(res_offset); + + prev_src_offset = curr_src_offset; + } +} + + +void registerFunctionArrayDistinct(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayElement.cpp b/dbms/src/Functions/arrayElement.cpp new file mode 100644 index 00000000000..ed3511fdb3e --- /dev/null +++ b/dbms/src/Functions/arrayElement.cpp @@ -0,0 +1,886 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ZERO_ARRAY_OR_TUPLE_INDEX; +} + +namespace ArrayImpl +{ + class NullMapBuilder; +} + +/** arrayElement(arr, i) - get the array element by index. If index is not constant and out of range - return default value of data type. + * The index begins with 1. Also, the index can be negative - then it is counted from the end of the array. + */ +class FunctionArrayElement : public IFunction +{ +public: + static constexpr auto name = "arrayElement"; + static FunctionPtr create(const Context & context); + + String getName() const override; + + bool useDefaultImplementationForConstants() const override { return true; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + void perform(Block & block, const ColumnNumbers & arguments, size_t result, + ArrayImpl::NullMapBuilder & builder, size_t input_rows_count); + + template + bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder); + + template + bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder); + + bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder); + + template + bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder); + + bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder); + + template + bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder); + + template + bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, + const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder, + size_t input_rows_count); + + template + bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, + ArrayImpl::NullMapBuilder & builder, size_t input_rows_count); + + /** For a tuple array, the function is evaluated component-wise for each element of the tuple. + */ + bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count); +}; + + +namespace ArrayImpl +{ + +class NullMapBuilder +{ +public: + operator bool() const { return src_null_map; } + bool operator!() const { return !src_null_map; } + + void initSource(const UInt8 * src_null_map_) + { + src_null_map = src_null_map_; + } + + void initSink(size_t size) + { + auto sink = ColumnUInt8::create(size); + sink_null_map = sink->getData().data(); + sink_null_map_holder = std::move(sink); + } + + void update(size_t from) + { + sink_null_map[index] = bool(src_null_map && src_null_map[from]); + ++index; + } + + void update() + { + sink_null_map[index] = bool(src_null_map); + ++index; + } + + ColumnPtr getNullMapColumnPtr() && { return std::move(sink_null_map_holder); } + +private: + const UInt8 * src_null_map = nullptr; + UInt8 * sink_null_map = nullptr; + MutableColumnPtr sink_null_map_holder; + size_t index = 0; +}; + +} + +namespace +{ + +template +struct ArrayElementNumImpl +{ + /** Implementation for constant index. + * If negative = false - index is from beginning of array, started from 0. + * If negative = true - index is from end of array, started from 0. + */ + template + static void vectorConst( + const PaddedPODArray & data, const ColumnArray::Offsets & offsets, + const ColumnArray::Offset index, + PaddedPODArray & result, ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result.resize(size); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + + if (index < array_size) + { + size_t j = !negative ? (current_offset + index) : (offsets[i] - index - 1); + result[i] = data[j]; + if (builder) + builder.update(j); + } + else + { + result[i] = T(); + + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } + + /** Implementation for non-constant index. + */ + template + static void vector( + const PaddedPODArray & data, const ColumnArray::Offsets & offsets, + const PaddedPODArray & indices, + PaddedPODArray & result, ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result.resize(size); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + + TIndex index = indices[i]; + if (index > 0 && static_cast(index) <= array_size) + { + size_t j = current_offset + index - 1; + result[i] = data[j]; + + if (builder) + builder.update(j); + } + else if (index < 0 && static_cast(-index) <= array_size) + { + size_t j = offsets[i] + index; + result[i] = data[j]; + + if (builder) + builder.update(j); + } + else + { + result[i] = T(); + + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } +}; + +struct ArrayElementStringImpl +{ + template + static void vectorConst( + const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const ColumnArray::Offset index, + ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, + ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result_offsets.resize(size); + result_data.reserve(data.size()); + + ColumnArray::Offset current_offset = 0; + ColumnArray::Offset current_result_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + + if (index < array_size) + { + size_t adjusted_index = !negative ? index : (array_size - index - 1); + + size_t j = current_offset + adjusted_index; + if (builder) + builder.update(j); + + ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0 + ? 0 + : string_offsets[current_offset + adjusted_index - 1]; + + ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos; + + result_data.resize(current_result_offset + string_size); + memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size); + current_result_offset += string_size; + result_offsets[i] = current_result_offset; + } + else + { + /// Insert an empty row. + result_data.resize(current_result_offset + 1); + result_data[current_result_offset] = 0; + current_result_offset += 1; + result_offsets[i] = current_result_offset; + + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } + + /** Implementation for non-constant index. + */ + template + static void vector( + const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const PaddedPODArray & indices, + ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, + ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result_offsets.resize(size); + result_data.reserve(data.size()); + + ColumnArray::Offset current_offset = 0; + ColumnArray::Offset current_result_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + size_t adjusted_index; /// index in array from zero + + TIndex index = indices[i]; + if (index > 0 && static_cast(index) <= array_size) + adjusted_index = index - 1; + else if (index < 0 && static_cast(-index) <= array_size) + adjusted_index = array_size + index; + else + adjusted_index = array_size; /// means no element should be taken + + if (adjusted_index < array_size) + { + size_t j = current_offset + adjusted_index; + if (builder) + builder.update(j); + + ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0 + ? 0 + : string_offsets[current_offset + adjusted_index - 1]; + + ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos; + + result_data.resize(current_result_offset + string_size); + memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size); + current_result_offset += string_size; + result_offsets[i] = current_result_offset; + } + else + { + /// Insert empty string + result_data.resize(current_result_offset + 1); + result_data[current_result_offset] = 0; + current_result_offset += 1; + result_offsets[i] = current_result_offset; + + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } +}; + +/// Generic implementation for other nested types. +struct ArrayElementGenericImpl +{ + template + static void vectorConst( + const IColumn & data, const ColumnArray::Offsets & offsets, + const ColumnArray::Offset index, + IColumn & result, ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result.reserve(size); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + + if (index < array_size) + { + size_t j = !negative ? current_offset + index : offsets[i] - index - 1; + result.insertFrom(data, j); + if (builder) + builder.update(j); + } + else + { + result.insertDefault(); + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } + + /** Implementation for non-constant index. + */ + template + static void vector( + const IColumn & data, const ColumnArray::Offsets & offsets, + const PaddedPODArray & indices, + IColumn & result, ArrayImpl::NullMapBuilder & builder) + { + size_t size = offsets.size(); + result.reserve(size); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + size_t array_size = offsets[i] - current_offset; + + TIndex index = indices[i]; + if (index > 0 && static_cast(index) <= array_size) + { + size_t j = current_offset + index - 1; + result.insertFrom(data, j); + if (builder) + builder.update(j); + } + else if (index < 0 && static_cast(-index) <= array_size) + { + size_t j = offsets[i] + index; + result.insertFrom(data, j); + if (builder) + builder.update(j); + } + else + { + result.insertDefault(); + if (builder) + builder.update(); + } + + current_offset = offsets[i]; + } + } +}; + +} + + +FunctionPtr FunctionArrayElement::create(const Context &) +{ + return std::make_shared(); +} + + +template +bool FunctionArrayElement::executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const ColumnVector * col_nested = checkAndGetColumn>(&col_array->getData()); + + if (!col_nested) + return false; + + auto col_res = ColumnVector::create(); + + if (index.getType() == Field::Types::UInt64) + ArrayElementNumImpl::template vectorConst( + col_nested->getData(), col_array->getOffsets(), safeGet(index) - 1, col_res->getData(), builder); + else if (index.getType() == Field::Types::Int64) + ArrayElementNumImpl::template vectorConst( + col_nested->getData(), col_array->getOffsets(), -safeGet(index) - 1, col_res->getData(), builder); + else + throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +template +bool FunctionArrayElement::executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const ColumnVector * col_nested = checkAndGetColumn>(&col_array->getData()); + + if (!col_nested) + return false; + + auto col_res = ColumnVector::create(); + + ArrayElementNumImpl::template vector( + col_nested->getData(), col_array->getOffsets(), indices, col_res->getData(), builder); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +bool FunctionArrayElement::executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const ColumnString * col_nested = checkAndGetColumn(&col_array->getData()); + + if (!col_nested) + return false; + + auto col_res = ColumnString::create(); + + if (index.getType() == Field::Types::UInt64) + ArrayElementStringImpl::vectorConst( + col_nested->getChars(), + col_array->getOffsets(), + col_nested->getOffsets(), + safeGet(index) - 1, + col_res->getChars(), + col_res->getOffsets(), + builder); + else if (index.getType() == Field::Types::Int64) + ArrayElementStringImpl::vectorConst( + col_nested->getChars(), + col_array->getOffsets(), + col_nested->getOffsets(), + -safeGet(index) - 1, + col_res->getChars(), + col_res->getOffsets(), + builder); + else + throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +template +bool FunctionArrayElement::executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const ColumnString * col_nested = checkAndGetColumn(&col_array->getData()); + + if (!col_nested) + return false; + + auto col_res = ColumnString::create(); + + ArrayElementStringImpl::vector( + col_nested->getChars(), + col_array->getOffsets(), + col_nested->getOffsets(), + indices, + col_res->getChars(), + col_res->getOffsets(), + builder); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +bool FunctionArrayElement::executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const auto & col_nested = col_array->getData(); + auto col_res = col_nested.cloneEmpty(); + + if (index.getType() == Field::Types::UInt64) + ArrayElementGenericImpl::vectorConst( + col_nested, col_array->getOffsets(), safeGet(index) - 1, *col_res, builder); + else if (index.getType() == Field::Types::Int64) + ArrayElementGenericImpl::vectorConst( + col_nested, col_array->getOffsets(), -safeGet(index) - 1, *col_res, builder); + else + throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +template +bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, + ArrayImpl::NullMapBuilder & builder) +{ + const ColumnArray * col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const auto & col_nested = col_array->getData(); + auto col_res = col_nested.cloneEmpty(); + + ArrayElementGenericImpl::vector( + col_nested, col_array->getOffsets(), indices, *col_res, builder); + + block.getByPosition(result).column = std::move(col_res); + return true; +} + +template +bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, + const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder, + size_t input_rows_count) +{ + const ColumnArray * col_array = checkAndGetColumnConstData(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + auto res = block.getByPosition(result).type->createColumn(); + + size_t rows = input_rows_count; + const IColumn & array_elements = col_array->getData(); + size_t array_size = array_elements.size(); + + for (size_t i = 0; i < rows; ++i) + { + IndexType index = indices[i]; + if (index > 0 && static_cast(index) <= array_size) + { + size_t j = index - 1; + res->insertFrom(array_elements, j); + if (builder) + builder.update(j); + } + else if (index < 0 && static_cast(-index) <= array_size) + { + size_t j = array_size + index; + res->insertFrom(array_elements, j); + if (builder) + builder.update(j); + } + else + { + res->insertDefault(); + if (builder) + builder.update(); + } + } + + block.getByPosition(result).column = std::move(res); + return true; +} + +template +bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, + ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) +{ + auto index = checkAndGetColumn>(block.getByPosition(arguments[1]).column.get()); + + if (!index) + return false; + + const auto & index_data = index->getData(); + + if (builder) + builder.initSink(index_data.size()); + + if (!( executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeNumber(block, arguments, result, index_data, builder) + || executeConst(block, arguments, result, index_data, builder, input_rows_count) + || executeString(block, arguments, result, index_data, builder) + || executeGeneric(block, arguments, result, index_data, builder))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + return true; +} + +bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + const ColumnArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); + + if (!col_array) + return false; + + const ColumnTuple * col_nested = typeid_cast(&col_array->getData()); + + if (!col_nested) + return false; + + const Columns & tuple_columns = col_nested->getColumns(); + size_t tuple_size = tuple_columns.size(); + + const DataTypes & tuple_types = typeid_cast( + *typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType()).getElements(); + + /** We will calculate the function for the tuple of the internals of the array. + * To do this, create a temporary block. + * It will consist of the following columns + * - the index of the array to be taken; + * - an array of the first elements of the tuples; + * - the result of taking the elements by the index for an array of the first elements of the tuples; + * - array of the second elements of the tuples; + * - result of taking elements by index for an array of second elements of tuples; + * ... + */ + Block block_of_temporary_results; + block_of_temporary_results.insert(block.getByPosition(arguments[1])); + + /// results of taking elements by index for arrays from each element of the tuples; + Columns result_tuple_columns; + + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnWithTypeAndName array_of_tuple_section; + array_of_tuple_section.column = ColumnArray::create(tuple_columns[i], col_array->getOffsetsPtr()); + array_of_tuple_section.type = std::make_shared(tuple_types[i]); + block_of_temporary_results.insert(array_of_tuple_section); + + ColumnWithTypeAndName array_elements_of_tuple_section; + array_elements_of_tuple_section.type = getReturnTypeImpl( + {block_of_temporary_results.getByPosition(i * 2 + 1).type, block_of_temporary_results.getByPosition(0).type}); + block_of_temporary_results.insert(array_elements_of_tuple_section); + + executeImpl(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2, input_rows_count); + + result_tuple_columns.emplace_back(std::move(block_of_temporary_results.getByPosition(i * 2 + 2).column)); + } + + block.getByPosition(result).column = ColumnTuple::create(result_tuple_columns); + + return true; +} + +String FunctionArrayElement::getName() const +{ + return name; +} + +DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) const +{ + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isInteger(arguments[1])) + throw Exception("Second argument for function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return array_type->getNestedType(); +} + +void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + /// Check nullability. + bool is_array_of_nullable = false; + + const ColumnArray * col_array = nullptr; + const ColumnArray * col_const_array = nullptr; + + col_array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + if (col_array) + is_array_of_nullable = col_array->getData().isColumnNullable(); + else + { + col_const_array = checkAndGetColumnConstData(block.getByPosition(arguments[0]).column.get()); + if (col_const_array) + is_array_of_nullable = col_const_array->getData().isColumnNullable(); + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + if (!is_array_of_nullable) + { + ArrayImpl::NullMapBuilder builder; + perform(block, arguments, result, builder, input_rows_count); + } + else + { + /// Perform initializations. + ArrayImpl::NullMapBuilder builder; + Block source_block; + + const auto & input_type = typeid_cast(*typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType(); + const auto & tmp_ret_type = typeid_cast(*block.getByPosition(result).type).getNestedType(); + + if (col_array) + { + const auto & nullable_col = typeid_cast(col_array->getData()); + const auto & nested_col = nullable_col.getNestedColumnPtr(); + + /// Put nested_col inside a ColumnArray. + source_block = + { + { + ColumnArray::create(nested_col, col_array->getOffsetsPtr()), + std::make_shared(input_type), + "" + }, + block.getByPosition(arguments[1]), + { + nullptr, + tmp_ret_type, + "" + } + }; + + builder.initSource(nullable_col.getNullMapData().data()); + } + else + { + /// ColumnConst(ColumnArray(ColumnNullable(...))) + const auto & nullable_col = static_cast(col_const_array->getData()); + const auto & nested_col = nullable_col.getNestedColumnPtr(); + + source_block = + { + { + ColumnConst::create(ColumnArray::create(nested_col, col_const_array->getOffsetsPtr()), input_rows_count), + std::make_shared(input_type), + "" + }, + block.getByPosition(arguments[1]), + { + nullptr, + tmp_ret_type, + "" + } + }; + + builder.initSource(nullable_col.getNullMapData().data()); + } + + perform(source_block, {0, 1}, 2, builder, input_rows_count); + + /// Store the result. + const ColumnWithTypeAndName & source_col = source_block.getByPosition(2); + ColumnWithTypeAndName & dest_col = block.getByPosition(result); + dest_col.column = ColumnNullable::create(source_col.column, builder ? std::move(builder).getNullMapColumnPtr() : ColumnUInt8::create()); + } +} + +void FunctionArrayElement::perform(Block & block, const ColumnNumbers & arguments, size_t result, + ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) +{ + if (executeTuple(block, arguments, result, input_rows_count)) + { + } + else if (!block.getByPosition(arguments[1]).column->isColumnConst()) + { + if (!(executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count) + || executeArgument(block, arguments, result, builder, input_rows_count))) + throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.", + ErrorCodes::ILLEGAL_COLUMN); + } + else + { + Field index = (*block.getByPosition(arguments[1]).column)[0]; + + if (builder) + builder.initSink(input_rows_count); + + if (index == UInt64(0)) + throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + + if (!( executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeNumberConst(block, arguments, result, index, builder) + || executeStringConst (block, arguments, result, index, builder) + || executeGenericConst (block, arguments, result, index, builder))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +} + + +void registerFunctionArrayElement(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayEnumerate.cpp b/dbms/src/Functions/arrayEnumerate.cpp new file mode 100644 index 00000000000..60fb41ca471 --- /dev/null +++ b/dbms/src/Functions/arrayEnumerate.cpp @@ -0,0 +1,85 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/// arrayEnumerate(arr) - Returns the array [1,2,3,..., length(arr)] +class FunctionArrayEnumerate : public IFunction +{ +public: + static constexpr auto name = "arrayEnumerate"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array but it has type " + + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + if (const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) + { + const ColumnArray::Offsets & offsets = array->getOffsets(); + + auto res_nested = ColumnUInt32::create(); + + ColumnUInt32::Container & res_values = res_nested->getData(); + res_values.resize(array->getData().size()); + ColumnArray::Offset prev_off = 0; + for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) + { + ColumnArray::Offset off = offsets[i]; + for (ColumnArray::Offset j = prev_off; j < off; ++j) + res_values[j] = j - prev_off + 1; + + prev_off = off; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr()); + } + else + { + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + } +}; + + +void registerFunctionArrayEnumerate(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayEnumerateDense.cpp b/dbms/src/Functions/arrayEnumerateDense.cpp new file mode 100644 index 00000000000..c4af9d905be --- /dev/null +++ b/dbms/src/Functions/arrayEnumerateDense.cpp @@ -0,0 +1,22 @@ +#include +#include + + +namespace DB +{ + + +class FunctionArrayEnumerateDense : public FunctionArrayEnumerateExtended +{ + using Base = FunctionArrayEnumerateExtended; +public: + static constexpr auto name = "arrayEnumerateDense"; + using Base::create; +}; + +void registerFunctionArrayEnumerateDense(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h new file mode 100644 index 00000000000..a88231d63ec --- /dev/null +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -0,0 +1,484 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; +} + +class FunctionArrayEnumerateUniq; +class FunctionArrayEnumerateDense; + +template +class FunctionArrayEnumerateExtended : public IFunction +{ +public: + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return Derived::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 + { + if (arguments.size() == 0) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < arguments.size(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception("All arguments for function " + getName() + " must be arrays but argument " + + toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return std::make_shared(std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. + static constexpr size_t INITIAL_SIZE_DEGREE = 9; + + template + bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + + bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + + bool execute128bit( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const ColumnRawPtrs & null_maps, + ColumnUInt32::Container & res_values, + bool has_nullable_columns); + + void executeHashed( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values); +}; + + +template +void FunctionArrayEnumerateExtended::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) +{ + const ColumnArray::Offsets * offsets = nullptr; + ColumnRawPtrs data_columns; + data_columns.reserve(arguments.size()); + + bool has_nullable_columns = false; + + for (size_t i = 0; i < arguments.size(); ++i) + { + ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; + const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + if (!array) + { + const ColumnConst * const_array = checkAndGetColumnConst( + block.getByPosition(arguments[i]).column.get()); + if (!const_array) + throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() + + " of " + toString(i + 1) + "-th argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + array_ptr = const_array->convertToFullColumn(); + array = checkAndGetColumn(array_ptr.get()); + } + + const ColumnArray::Offsets & offsets_i = array->getOffsets(); + if (i == 0) + offsets = &offsets_i; + else if (offsets_i != *offsets) + throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + + auto * array_data = &array->getData(); + data_columns.push_back(array_data); + } + + size_t num_columns = data_columns.size(); + ColumnRawPtrs original_data_columns(num_columns); + ColumnRawPtrs null_maps(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + original_data_columns[i] = data_columns[i]; + + if (data_columns[i]->isColumnNullable()) + { + has_nullable_columns = true; + const auto & nullable_col = static_cast(*data_columns[i]); + data_columns[i] = &nullable_col.getNestedColumn(); + null_maps[i] = &nullable_col.getNullMapColumn(); + } + else + null_maps[i] = nullptr; + } + + const ColumnArray * first_array = checkAndGetColumn(block.getByPosition(arguments.at(0)).column.get()); + const IColumn * first_null_map = null_maps[0]; + auto res_nested = ColumnUInt32::create(); + + ColumnUInt32::Container & res_values = res_nested->getData(); + if (!offsets->empty()) + res_values.resize(offsets->back()); + + if (num_columns == 1) + { + if (!( executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeString (first_array, first_null_map, res_values))) + executeHashed(*offsets, original_data_columns, res_values); + } + else + { + if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) + executeHashed(*offsets, original_data_columns, res_values); + } + + block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), first_array->getOffsetsPtr()); +} + + +template +template +bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +{ + const IColumn * inner_col; + + const auto & array_data = array->getData(); + if (array_data.isColumnNullable()) + { + const auto & nullable_col = static_cast(array_data); + inner_col = &nullable_col.getNestedColumn(); + } + else + inner_col = &array_data; + + const ColumnVector * nested = checkAndGetColumn>(inner_col); + if (!nested) + return false; + const ColumnArray::Offsets & offsets = array->getOffsets(); + const typename ColumnVector::Container & values = nested->getData(); + + using ValuesToIndices = ClearableHashMap, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + + const PaddedPODArray * null_map_data = nullptr; + if (null_map) + null_map_data = &static_cast(null_map)->getData(); + + ValuesToIndices indices; + size_t prev_off = 0; + if constexpr (std::is_same_v) + { + // Unique + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + UInt32 null_count = 0; + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + res_values[j] = ++null_count; + else + res_values[j] = ++indices[values[j]]; + } + prev_off = off; + } + } + else + { + // Dense + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t rank = 0; + UInt32 null_index = 0; + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + { + if (!null_index) + null_index = ++rank; + res_values[j] = null_index; + } + else + { + auto & idx = indices[values[j]]; + if (!idx) + idx = ++rank; + res_values[j] = idx; + } + } + prev_off = off; + } + } + return true; +} + +template +bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +{ + const IColumn * inner_col; + + const auto & array_data = array->getData(); + if (array_data.isColumnNullable()) + { + const auto & nullable_col = static_cast(array_data); + inner_col = &nullable_col.getNestedColumn(); + } + else + inner_col = &array_data; + + const ColumnString * nested = checkAndGetColumn(inner_col); + if (!nested) + return false; + const ColumnArray::Offsets & offsets = array->getOffsets(); + + size_t prev_off = 0; + using ValuesToIndices = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + + const PaddedPODArray * null_map_data = nullptr; + if (null_map) + null_map_data = &static_cast(null_map)->getData(); + + ValuesToIndices indices; + if constexpr (std::is_same_v) + { + // Unique + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + UInt32 null_count = 0; + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + res_values[j] = ++null_count; + else + res_values[j] = ++indices[nested->getDataAt(j)]; + } + prev_off = off; + } + } + else + { + // Dense + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t rank = 0; + UInt32 null_index = 0; + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + { + if (!null_index) + null_index = ++rank; + res_values[j] = null_index; + } + else + { + auto & idx = indices[nested->getDataAt(j)]; + if (!idx) + idx = ++rank; + res_values[j] = idx; + } + } + prev_off = off; + } + } + return true; +} + +template +bool FunctionArrayEnumerateExtended::execute128bit( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const ColumnRawPtrs & null_maps, + ColumnUInt32::Container & res_values, + bool has_nullable_columns) +{ + size_t count = columns.size(); + size_t keys_bytes = 0; + Sizes key_sizes(count); + + for (size_t j = 0; j < count; ++j) + { + if (!columns[j]->isFixedAndContiguous()) + return false; + key_sizes[j] = columns[j]->sizeOfValueIfFixed(); + keys_bytes += key_sizes[j]; + } + if (has_nullable_columns) + keys_bytes += std::tuple_size>::value; + + if (keys_bytes > 16) + return false; + + using ValuesToIndices = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + + ValuesToIndices indices; + size_t prev_off = 0; + if constexpr (std::is_same_v) + { + // Unique + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + if (has_nullable_columns) + { + KeysNullMap bitmap{}; + + for (size_t i = 0; i < columns.size(); ++i) + { + if (null_maps[i]) + { + const auto & null_map = static_cast(*null_maps[i]).getData(); + if (null_map[j] == 1) + { + size_t bucket = i / 8; + size_t offset = i % 8; + bitmap[bucket] |= UInt8(1) << offset; + } + } + } + res_values[j] = ++indices[packFixed(j, count, columns, key_sizes, bitmap)]; + } + else + res_values[j] = ++indices[packFixed(j, count, columns, key_sizes)]; + } + prev_off = off; + } + } + else + { + // Dense + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t off = offsets[i]; + size_t rank = 0; + for (size_t j = prev_off; j < off; ++j) + { + if (has_nullable_columns) + { + KeysNullMap bitmap{}; + + for (size_t i = 0; i < columns.size(); ++i) + { + if (null_maps[i]) + { + const auto & null_map = static_cast(*null_maps[i]).getData(); + if (null_map[j] == 1) + { + size_t bucket = i / 8; + size_t offset = i % 8; + bitmap[bucket] |= UInt8(1) << offset; + } + } + } + auto &idx = indices[packFixed(j, count, columns, key_sizes, bitmap)]; + if (!idx) + idx = ++rank; + res_values[j] = idx; + } + else + { + auto &idx = indices[packFixed(j, count, columns, key_sizes)];; + if (!idx) + idx = ++rank; + res_values[j] = idx; + } + } + prev_off = off; + } + } + + return true; +} + +template +void FunctionArrayEnumerateExtended::executeHashed( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values) +{ + size_t count = columns.size(); + + using ValuesToIndices = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + + ValuesToIndices indices; + size_t prev_off = 0; + if constexpr (std::is_same_v) + { + // Unique + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t off = offsets[i]; + for (size_t j = prev_off; j < off; ++j) + { + res_values[j] = ++indices[hash128(j, count, columns)]; + } + prev_off = off; + } + } + else + { + // Dense + for (size_t i = 0; i < offsets.size(); ++i) + { + indices.clear(); + size_t off = offsets[i]; + size_t rank = 0; + for (size_t j = prev_off; j < off; ++j) + { + auto & idx = indices[hash128(j, count, columns)]; + if (!idx) + idx = ++rank; + res_values[j] = idx; + } + prev_off = off; + } + } +} + +} diff --git a/dbms/src/Functions/arrayEnumerateUniq.cpp b/dbms/src/Functions/arrayEnumerateUniq.cpp new file mode 100644 index 00000000000..211c29b3435 --- /dev/null +++ b/dbms/src/Functions/arrayEnumerateUniq.cpp @@ -0,0 +1,28 @@ +#include +#include + + +namespace DB +{ + +/** arrayEnumerateUniq(arr) + * - outputs an array parallel (having same size) to this, where for each element specified + * how many times this element was encountered before (including this element) among elements with the same value. + * For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1] + * arrayEnumerateUniq(arr1, arr2...) + * - for tuples from elements in the corresponding positions in several arrays. + */ +class FunctionArrayEnumerateUniq : public FunctionArrayEnumerateExtended +{ + using Base = FunctionArrayEnumerateExtended; +public: + static constexpr auto name = "arrayEnumerateUniq"; + using Base::create; +}; + +void registerFunctionArrayEnumerateUniq(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/FunctionsArray.h b/dbms/src/Functions/arrayIndex.h similarity index 57% rename from dbms/src/Functions/FunctionsArray.h rename to dbms/src/Functions/arrayIndex.h index 5b12e7965b8..68589f56f3e 100644 --- a/dbms/src/Functions/FunctionsArray.h +++ b/dbms/src/Functions/arrayIndex.h @@ -1,27 +1,14 @@ -#pragma once - -#include - +#include +#include +#include #include -#include -#include -#include -#include #include - +#include #include #include -#include +#include #include - -#include -#include -#include - -#include - -#include -#include +#include namespace DB @@ -30,141 +17,10 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } - -/** Array functions: - * - * array(c1, c2, ...) - create an array. - * arrayElement(arr, i) - get the array element by index. If index is not constant and out of range - return default value of data type. - * The index begins with 1. Also, the index can be negative - then it is counted from the end of the array. - * has(arr, x) - whether there is an element x in the array. - * indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not. - * arrayEnumerate(arr) - Returns the array [1,2,3,..., length(arr)] - * - * arrayUniq(arr) - counts the number of different elements in the array, - * arrayUniq(arr1, arr2, ...) - counts the number of different tuples from the elements in the corresponding positions in several arrays. - * - * arrayDistinct(arr) - retrun different elements in an array - * - * arrayEnumerateUniq(arr) - * - outputs an array parallel (having same size) to this, where for each element specified - * how many times this element was encountered before (including this element) among elements with the same value. - * For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1] - * arrayEnumerateUniq(arr1, arr2...) - * - for tuples from elements in the corresponding positions in several arrays. - * - * emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value. - * - * arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...` - * If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function. - * - * arrayConcat(arr1, ...) - concatenate arrays. - * - * arraySlice(arr, offset, length) - make slice of array. Offsets and length may be < 0 or Null - * - if offset < 0, indexation from right element - * - if length < 0, length = len(array) - (positive_index(offset) - 1) + length - * indexation: - * [ 1, 2, 3, 4, 5, 6] - * [-6, -5, -4, -3, -2, -1] - * examples: - * arraySlice([1, 2, 3, 4, 5, 6], -4, 2) -> [3, 4] - * arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] (6 - (2 - 1) + (-1) = 4) - * arraySlice([1, 2, 3, 4, 5, 6], -5, -1) = arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] - * - * arrayPushBack(arr, x), arrayPushFront(arr, x) - * arrayPopBack(arr), arrayPopFront(arr) - */ - - -class FunctionArray : public IFunction -{ -public: - static constexpr auto name = "array"; - static FunctionPtr create(const Context & context); - - FunctionArray(const Context & context); - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - String getName() const override; - - bool addField(DataTypePtr type_res, const Field & f, Array & arr) const; - -private: - const Context & context; -}; - -namespace ArrayImpl -{ - class NullMapBuilder; -} - -class FunctionArrayElement : public IFunction -{ -public: - static constexpr auto name = "arrayElement"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - bool useDefaultImplementationForConstants() const override { return true; } - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - void perform(Block & block, const ColumnNumbers & arguments, size_t result, - ArrayImpl::NullMapBuilder & builder, size_t input_rows_count); - - template - bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder); - - template - bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder); - - bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder); - - template - bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder); - - bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, - ArrayImpl::NullMapBuilder & builder); - - template - bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, - ArrayImpl::NullMapBuilder & builder); - - template - bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder, - size_t input_rows_count); - - template - bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, - ArrayImpl::NullMapBuilder & builder, size_t input_rows_count); - - /** For a tuple array, the function is evaluated component-wise for each element of the tuple. - */ - bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count); -}; - - /// For has. struct IndexToOne { @@ -734,9 +590,12 @@ struct ArrayIndexGenericNullImpl }; -inline bool allowArrayIndex(const IDataType * data_type0, const IDataType * data_type1) +inline bool allowArrayIndex(const DataTypePtr & type0, const DataTypePtr & type1) { - return ((data_type0->isNumber() || isEnum(data_type0)) && data_type1->isNumber()) + DataTypePtr data_type0 = removeNullable(type0); + DataTypePtr data_type1 = removeNullable(type1); + + return ((isNumber(data_type0) || isEnum(data_type0)) && isNumber(data_type1)) || data_type0->equals(*data_type1); } @@ -1015,10 +874,7 @@ public: if (!arguments[1]->onlyNull()) { - DataTypePtr observed_type0 = removeNullable(array_type->getNestedType()); - DataTypePtr observed_type1 = removeNullable(arguments[1]); - - if (!allowArrayIndex(observed_type0.get(), observed_type1.get())) + if (!allowArrayIndex(array_type->getNestedType(), arguments[1])) throw Exception("Types of array and 2nd argument of function " + getName() + " must be identical up to nullability or numeric types or Enum and numeric type. Passed: " + arguments[0]->getName() + " and " + arguments[1]->getName() + ".", @@ -1159,587 +1015,4 @@ private: } }; - -class FunctionArrayEnumerate : public IFunction -{ -public: - static constexpr auto name = "arrayEnumerate"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - - -/// Counts the number of different elements in the array, or the number of different tuples from the elements at the corresponding positions in several arrays. -/// NOTE The implementation partially matches arrayEnumerateUniq. -class FunctionArrayUniq : public IFunction -{ -public: - static constexpr auto name = "arrayUniq"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - 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; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. - static constexpr size_t INITIAL_SIZE_DEGREE = 9; - - template - bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns); - - void executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values); -}; - - -/// Find different elements in an array. -class FunctionArrayDistinct : public IFunction -{ -public: - static constexpr auto name = "arrayDistinct"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - bool isVariadic() const override { return false; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. - static constexpr size_t INITIAL_SIZE_DEGREE = 9; - - template - bool executeNumber( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col); - - bool executeString( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col); - - void executeHashed( - const IColumn & src_data, - const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - ColumnArray::Offsets & res_offsets, - const ColumnNullable * nullable_col); -}; - - -template -class FunctionArrayEnumerateExtended : public IFunction -{ -public: - static FunctionPtr create(const Context & context); - - String getName() const override; - - 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; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. - static constexpr size_t INITIAL_SIZE_DEGREE = 9; - - template - bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns); - - void executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values); -}; - -class FunctionArrayEnumerateUniq : public FunctionArrayEnumerateExtended -{ - using Base = FunctionArrayEnumerateExtended; -public: - static constexpr auto name = "arrayEnumerateUniq"; - using Base::create; -}; - -class FunctionArrayEnumerateDense : public FunctionArrayEnumerateExtended -{ - using Base = FunctionArrayEnumerateExtended; -public: - static constexpr auto name = "arrayEnumerateDense"; - using Base::create; -}; - -template struct TypeToColumnType { using ColumnType = ColumnVector; }; -template <> struct TypeToColumnType { using ColumnType = ColumnString; }; - -template struct DataTypeToName : TypeName { }; -template <> struct DataTypeToName { static std::string get() { return "Date"; } }; -template <> struct DataTypeToName { static std::string get() { return "DateTime"; } }; - -template -struct FunctionEmptyArray : public IFunction -{ - static constexpr auto base_name = "emptyArray"; - static const String name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - -private: - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(std::make_shared()); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - using UnderlyingColumnType = typename TypeToColumnType::ColumnType; - - block.getByPosition(result).column = ColumnArray::create( - UnderlyingColumnType::create(), - ColumnArray::ColumnOffsets::create(input_rows_count, 0)); - } -}; - -template -const String FunctionEmptyArray::name = FunctionEmptyArray::base_name + String(DataTypeToName::get()); - -class FunctionRange : public IFunction -{ -public: - static constexpr auto name = "range"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - -private: - String getName() const override; - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - template - bool executeInternal(Block & block, const IColumn * arg, const size_t result); - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - - -class FunctionEmptyArrayToSingle : public IFunction -{ -public: - static constexpr auto name = "emptyArrayToSingle"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - - -class FunctionArrayReverse : public IFunction -{ -public: - static constexpr auto name = "arrayReverse"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - size_t input_rows_count); - - template - bool executeNumber( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); - - bool executeFixedString( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); - - bool executeString( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); -}; - - -class IAggregateFunction; -using AggregateFunctionPtr = std::shared_ptr; - -/** Applies an aggregate function to array and returns its result. - * If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size. - */ -class FunctionArrayReduce : public IFunction -{ -public: - static constexpr auto name = "arrayReduce"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -private: - /// lazy initialization in getReturnTypeImpl - /// TODO: init in FunctionBuilder - mutable AggregateFunctionPtr aggregate_function; -}; - - -class FunctionArrayConcat : public IFunction -{ -public: - static constexpr auto name = "arrayConcat"; - static FunctionPtr create(const Context & context); - FunctionArrayConcat(const Context & context) : context(context) {} - - String getName() const override; - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - -private: - const Context & context; -}; - - -class FunctionArraySlice : public IFunction -{ -public: - static constexpr auto name = "arraySlice"; - static FunctionPtr create(const Context & context); - - String getName() const override; - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return false; } -}; - - -class FunctionArrayPush : public IFunction -{ -public: - FunctionArrayPush(const Context & context, bool push_front, const char * name) - : context(context), push_front(push_front), name(name) {} - - String getName() const override { return name; } - - bool isVariadic() const override { return false; } - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return false; } - -private: - const Context & context; - bool push_front; - const char * name; -}; - -class FunctionArrayPushFront : public FunctionArrayPush -{ -public: - static constexpr auto name = "arrayPushFront"; - - static FunctionPtr create(const Context & context); - FunctionArrayPushFront(const Context & context) : FunctionArrayPush(context, true, name) {} -}; - -class FunctionArrayPushBack : public FunctionArrayPush -{ -public: - static constexpr auto name = "arrayPushBack"; - - static FunctionPtr create(const Context & context); - FunctionArrayPushBack(const Context & context) : FunctionArrayPush(context, false, name) {} -}; - -class FunctionArrayPop : public IFunction -{ -public: - FunctionArrayPop(bool pop_front, const char * name) : pop_front(pop_front), name(name) {} - - String getName() const override { return name; } - - bool isVariadic() const override { return false; } - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return false; } - -private: - bool pop_front; - const char * name; -}; - -class FunctionArrayPopFront : public FunctionArrayPop -{ -public: - static constexpr auto name = "arrayPopFront"; - - static FunctionPtr create(const Context & context); - - FunctionArrayPopFront() : FunctionArrayPop(true, name) {} -}; - -class FunctionArrayPopBack : public FunctionArrayPop -{ -public: - static constexpr auto name = "arrayPopBack"; - - static FunctionPtr create(const Context & context); - - FunctionArrayPopBack() : FunctionArrayPop(false, name) {} -}; - -class FunctionArrayIntersect : public IFunction -{ -public: - static constexpr auto name = "arrayIntersect"; - static FunctionPtr create(const Context & context); - FunctionArrayIntersect(const Context & context) : context(context) {} - - String getName() const override; - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - -private: - const Context & context; - - /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. - static constexpr size_t INITIAL_SIZE_DEGREE = 9; - - struct UnpackedArrays - { - std::vector is_const; - std::vector null_maps; - std::vector offsets; - ColumnRawPtrs nested_columns; - - UnpackedArrays() = default; - }; - - /// Cast column to data_type removing nullable if data_type hasn't. - /// It's expected that column can represent data_type after removing some NullMap's. - ColumnPtr castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const; - Columns castColumns(Block & block, const ColumnNumbers & arguments, - const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const; - UnpackedArrays prepareArrays(const Columns & columns) const; - - template - static ColumnPtr execute(const UnpackedArrays & arrays, MutableColumnPtr result_data); - - struct NumberExecutor - { - const UnpackedArrays & arrays; - const DataTypePtr & data_type; - ColumnPtr & result; - - NumberExecutor(const UnpackedArrays & arrays, const DataTypePtr & data_type, ColumnPtr & result) - : arrays(arrays), data_type(data_type), result(result) {} - - template - void operator()(); - }; -}; - -class FunctionArrayHasAllAny : public IFunction -{ -public: - FunctionArrayHasAllAny(const Context & context, bool all, const char * name) - : context(context), all(all), name(name) {} - - String getName() const override { return name; } - - bool isVariadic() const override { return false; } - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - -private: - const Context & context; - bool all; - const char * name; -}; - -class FunctionArrayHasAll : public FunctionArrayHasAllAny -{ -public: - static constexpr auto name = "hasAll"; - - static FunctionPtr create(const Context & context); - - FunctionArrayHasAll(const Context & context) : FunctionArrayHasAllAny(context, true, name) {} -}; - -class FunctionArrayHasAny : public FunctionArrayHasAllAny -{ -public: - static constexpr auto name = "hasAny"; - - static FunctionPtr create(const Context & context); - - FunctionArrayHasAny(const Context & context) : FunctionArrayHasAllAny(context, false, name) {} -}; - - -class FunctionArrayResize : public IFunction -{ -public: - static constexpr auto name = "arrayResize"; - static FunctionPtr create(const Context & context); - FunctionArrayResize(const Context & context) : context(context) {} - - String getName() const override; - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return false; } - -private: - const Context & context; -}; - -struct NameHas { static constexpr auto name = "has"; }; -struct NameIndexOf { static constexpr auto name = "indexOf"; }; -struct NameCountEqual { static constexpr auto name = "countEqual"; }; - -using FunctionHas = FunctionArrayIndex; -using FunctionIndexOf = FunctionArrayIndex; -using FunctionCountEqual = FunctionArrayIndex; - -using FunctionEmptyArrayUInt8 = FunctionEmptyArray; -using FunctionEmptyArrayUInt16 = FunctionEmptyArray; -using FunctionEmptyArrayUInt32 = FunctionEmptyArray; -using FunctionEmptyArrayUInt64 = FunctionEmptyArray; -using FunctionEmptyArrayInt8 = FunctionEmptyArray; -using FunctionEmptyArrayInt16 = FunctionEmptyArray; -using FunctionEmptyArrayInt32 = FunctionEmptyArray; -using FunctionEmptyArrayInt64 = FunctionEmptyArray; -using FunctionEmptyArrayFloat32 = FunctionEmptyArray; -using FunctionEmptyArrayFloat64 = FunctionEmptyArray; -using FunctionEmptyArrayDate = FunctionEmptyArray; -using FunctionEmptyArrayDateTime = FunctionEmptyArray; -using FunctionEmptyArrayString = FunctionEmptyArray; - - } diff --git a/dbms/src/Functions/arrayIntersect.cpp b/dbms/src/Functions/arrayIntersect.cpp new file mode 100644 index 00000000000..85678c10f1d --- /dev/null +++ b/dbms/src/Functions/arrayIntersect.cpp @@ -0,0 +1,457 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionArrayIntersect : public IFunction +{ +public: + static constexpr auto name = "arrayIntersect"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayIntersect(const Context & context) : context(context) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + + bool useDefaultImplementationForConstants() const override { return true; } + +private: + const Context & context; + + /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. + static constexpr size_t INITIAL_SIZE_DEGREE = 9; + + struct UnpackedArrays + { + std::vector is_const; + std::vector null_maps; + std::vector offsets; + ColumnRawPtrs nested_columns; + + UnpackedArrays() = default; + }; + + /// Cast column to data_type removing nullable if data_type hasn't. + /// It's expected that column can represent data_type after removing some NullMap's. + ColumnPtr castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const; + Columns castColumns(Block & block, const ColumnNumbers & arguments, + const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const; + UnpackedArrays prepareArrays(const Columns & columns) const; + + template + static ColumnPtr execute(const UnpackedArrays & arrays, MutableColumnPtr result_data); + + struct NumberExecutor + { + const UnpackedArrays & arrays; + const DataTypePtr & data_type; + ColumnPtr & result; + + NumberExecutor(const UnpackedArrays & arrays, const DataTypePtr & data_type, ColumnPtr & result) + : arrays(arrays), data_type(data_type), result(result) {} + + template + void operator()(); + }; +}; + + +DataTypePtr FunctionArrayIntersect::getReturnTypeImpl(const DataTypes & arguments) const +{ + DataTypes nested_types; + nested_types.reserve(arguments.size()); + + bool has_nothing = false; + + if (arguments.empty()) + throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + for (auto i : ext::range(0, arguments.size())) + { + auto array_type = typeid_cast(arguments[i].get()); + if (!array_type) + throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " + + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto & nested_type = array_type->getNestedType(); + + if (typeid_cast(nested_type.get())) + has_nothing = true; + else + nested_types.push_back(nested_type); + } + + DataTypePtr result_type; + + if (!nested_types.empty()) + result_type = getMostSubtype(nested_types, true); + + if (has_nothing) + result_type = std::make_shared(); + + return std::make_shared(result_type); +} + +ColumnPtr FunctionArrayIntersect::castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const +{ + if (auto column_nullable = checkAndGetColumn(column.get())) + { + auto nullable_type = checkAndGetDataType(data_type.get()); + const auto & nested = column_nullable->getNestedColumnPtr(); + if (nullable_type) + { + auto casted_column = castRemoveNullable(nested, nullable_type->getNestedType()); + return ColumnNullable::create(casted_column, column_nullable->getNullMapColumnPtr()); + } + return castRemoveNullable(nested, data_type); + } + else if (auto column_array = checkAndGetColumn(column.get())) + { + auto array_type = checkAndGetDataType(data_type.get()); + if (!array_type) + throw Exception{"Cannot cast array column to column with type " + + data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR}; + + auto casted_column = castRemoveNullable(column_array->getDataPtr(), array_type->getNestedType()); + return ColumnArray::create(casted_column, column_array->getOffsetsPtr()); + } + else if (auto column_tuple = checkAndGetColumn(column.get())) + { + auto tuple_type = checkAndGetDataType(data_type.get()); + + if (!tuple_type) + throw Exception{"Cannot cast tuple column to type " + + data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR}; + + auto columns_number = column_tuple->getColumns().size(); + Columns columns(columns_number); + + const auto & types = tuple_type->getElements(); + + for (auto i : ext::range(0, columns_number)) + { + columns[i] = castRemoveNullable(column_tuple->getColumnPtr(i), types[i]); + } + return ColumnTuple::create(columns); + } + + return column; +} + +Columns FunctionArrayIntersect::castColumns( + Block & block, const ColumnNumbers & arguments, const DataTypePtr & return_type, + const DataTypePtr & return_type_with_nulls) const +{ + size_t num_args = arguments.size(); + Columns columns(num_args); + + auto type_array = checkAndGetDataType(return_type.get()); + auto & type_nested = type_array->getNestedType(); + auto type_not_nullable_nested = removeNullable(type_nested); + + const bool is_numeric_or_string = isNumber(type_not_nullable_nested) + || isDateOrDateTime(type_not_nullable_nested) + || isStringOrFixedString(type_not_nullable_nested); + + DataTypePtr nullable_return_type; + + if (is_numeric_or_string) + { + auto type_nullable_nested = makeNullable(type_nested); + nullable_return_type = std::make_shared(type_nullable_nested); + } + + const bool nested_is_nullable = type_nested->isNullable(); + + for (size_t i = 0; i < num_args; ++i) + { + const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]); + auto & column = columns[i]; + + if (is_numeric_or_string) + { + /// Cast to Array(T) or Array(Nullable(T)). + if (nested_is_nullable) + { + if (arg.type->equals(*return_type)) + column = arg.column; + else + column = castColumn(arg, return_type, context); + } + else + { + /// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T)) + /// because cannot cast Nullable(T) to T. + if (arg.type->equals(*return_type) || arg.type->equals(*nullable_return_type)) + column = arg.column; + else if (static_cast(*arg.type).getNestedType()->isNullable()) + column = castColumn(arg, nullable_return_type, context); + else + column = castColumn(arg, return_type, context); + } + } + else + { + /// return_type_with_nulls is the most common subtype with possible nullable parts. + if (arg.type->equals(*return_type_with_nulls)) + column = arg.column; + else + column = castColumn(arg, return_type_with_nulls, context); + } + } + + return columns; +} + +FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(const Columns & columns) const +{ + UnpackedArrays arrays; + + size_t columns_number = columns.size(); + arrays.is_const.assign(columns_number, false); + arrays.null_maps.resize(columns_number); + arrays.offsets.resize(columns_number); + arrays.nested_columns.resize(columns_number); + + for (auto i : ext::range(0, columns_number)) + { + auto argument_column = columns[i].get(); + if (auto argument_column_const = typeid_cast(argument_column)) + { + arrays.is_const[i] = true; + argument_column = argument_column_const->getDataColumnPtr().get(); + } + + if (auto argument_column_array = typeid_cast(argument_column)) + { + arrays.offsets[i] = &argument_column_array->getOffsets(); + arrays.nested_columns[i] = &argument_column_array->getData(); + if (auto column_nullable = typeid_cast(arrays.nested_columns[i])) + { + arrays.null_maps[i] = &column_nullable->getNullMapData(); + arrays.nested_columns[i] = &column_nullable->getNestedColumn(); + } + } + else + throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; + } + + return arrays; +} + +void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + const auto & return_type = block.getByPosition(result).type; + auto return_type_array = checkAndGetDataType(return_type.get()); + + if (!return_type) + throw Exception{"Return type for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; + + const auto & nested_return_type = return_type_array->getNestedType(); + + if (typeid_cast(nested_return_type.get())) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto num_args = arguments.size(); + DataTypes data_types; + data_types.reserve(num_args); + for (size_t i = 0; i < num_args; ++i) + data_types.push_back(block.getByPosition(arguments[i]).type); + + auto return_type_with_nulls = getMostSubtype(data_types, true, true); + + Columns columns = castColumns(block, arguments, return_type, return_type_with_nulls); + + UnpackedArrays arrays = prepareArrays(columns); + + ColumnPtr result_column; + auto not_nullable_nested_return_type = removeNullable(nested_return_type); + TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column)); + + using DateMap = ClearableHashMap, + HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDate::FieldType)>>; + + using DateTimeMap = ClearableHashMap, + HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDateTime::FieldType)>>; + + using StringMap = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + + if (!result_column) + { + auto column = not_nullable_nested_return_type->createColumn(); + WhichDataType which(not_nullable_nested_return_type); + + if (which.isDate()) + result_column = execute, true>(arrays, std::move(column)); + else if (which.isDateTime()) + result_column = execute, true>(arrays, std::move(column)); + else if(which.isString()) + result_column = execute(arrays, std::move(column)); + else if(which.isFixedString()) + result_column = execute(arrays, std::move(column)); + else + { + column = static_cast(*return_type_with_nulls).getNestedType()->createColumn(); + result_column = castRemoveNullable(execute(arrays, std::move(column)), return_type); + } + } + + block.getByPosition(result).column = std::move(result_column); +} + +template +void FunctionArrayIntersect::NumberExecutor::operator()() +{ + using Map = ClearableHashMap, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + + if (!result && typeid_cast *>(data_type.get())) + result = execute, true>(arrays, ColumnVector::create()); +} + +template +ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr) +{ + auto args = arrays.nested_columns.size(); + auto rows = arrays.offsets.front()->size(); + + bool all_nullable = true; + + std::vector columns; + columns.reserve(args); + for (auto arg : ext::range(0, args)) + { + if constexpr (std::is_same::value) + columns.push_back(arrays.nested_columns[arg]); + else + columns.push_back(checkAndGetColumn(arrays.nested_columns[arg])); + + if (!columns.back()) + throw Exception("Unexpected array type for function arrayIntersect", ErrorCodes::LOGICAL_ERROR); + + if (!arrays.null_maps[arg]) + all_nullable = false; + } + + auto & result_data = static_cast(*result_data_ptr); + auto result_offsets_ptr = ColumnArray::ColumnOffsets::create(rows); + auto & result_offsets = static_cast(*result_offsets_ptr); + auto null_map_column = ColumnUInt8::create(); + NullMap & null_map = static_cast(*null_map_column).getData(); + + Arena arena; + + Map map; + std::vector prev_off(args, 0); + size_t result_offset = 0; + for (auto row : ext::range(0, rows)) + { + map.clear(); + + bool all_has_nullable = all_nullable; + + for (auto arg : ext::range(0, args)) + { + bool current_has_nullable = false; + size_t off = (*arrays.offsets[arg])[row]; + for (auto i : ext::range(prev_off[arg], off)) + { + if (arrays.null_maps[arg] && (*arrays.null_maps[arg])[i]) + current_has_nullable = true; + else + { + if constexpr (is_numeric_column) + ++map[columns[arg]->getElement(i)]; + else if constexpr (std::is_same::value || std::is_same::value) + ++map[columns[arg]->getDataAt(i)]; + else + { + const char * data = nullptr; + ++map[columns[arg]->serializeValueIntoArena(i, arena, data)]; + } + } + } + + prev_off[arg] = off; + if (!current_has_nullable) + all_has_nullable = false; + } + + if (all_has_nullable) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + } + + for (const auto & pair : map) + { + if (pair.second == args) + { + ++result_offset; + if constexpr (is_numeric_column) + result_data.insert(pair.first); + else if constexpr (std::is_same::value || std::is_same::value) + result_data.insertData(pair.first.data, pair.first.size); + else + result_data.deserializeAndInsertFromArena(pair.first.data); + + if (all_nullable) + null_map.push_back(0); + } + } + result_offsets.getElement(row) = result_offset; + } + + ColumnPtr result_column = std::move(result_data_ptr); + if (all_nullable) + result_column = ColumnNullable::create(result_column, std::move(null_map_column)); + return ColumnArray::create(result_column, std::move(result_offsets_ptr)); +} + + +void registerFunctionArrayIntersect(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayJoin.cpp b/dbms/src/Functions/arrayJoin.cpp new file mode 100644 index 00000000000..7e7dba8e71a --- /dev/null +++ b/dbms/src/Functions/arrayJoin.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FUNCTION_IS_SPECIAL; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** arrayJoin(arr) - a special function - it can not be executed directly; + * is used only to get the result type of the corresponding expression. + */ +class FunctionArrayJoin : public IFunction +{ +public: + static constexpr auto name = "arrayJoin"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + /** It could return many different values for single argument. */ + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * arr = checkAndGetDataType(arguments[0].get()); + if (!arr) + throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arr->getNestedType(); + } + + void executeImpl(Block &, const ColumnNumbers &, size_t, size_t /*input_rows_count*/) override + { + throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL); + } + + /// Because of function cannot be executed directly. + bool isSuitableForConstantFolding() const override + { + return false; + } +}; + + +void registerFunctionArrayJoin(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayPop.h b/dbms/src/Functions/arrayPop.h new file mode 100644 index 00000000000..0336d2e20eb --- /dev/null +++ b/dbms/src/Functions/arrayPop.h @@ -0,0 +1,82 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionArrayPop : public IFunction +{ +public: + FunctionArrayPop(bool pop_front, const char * name) : pop_front(pop_front), name(name) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments[0]->onlyNull()) + return arguments[0]; + + auto array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array but it has type " + + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & return_type = block.getByPosition(result).type; + + if (return_type->onlyNull()) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto result_column = return_type->createColumn(); + + const auto & array_column = block.getByPosition(arguments[0]).column; + + std::unique_ptr source; + + size_t size = array_column->size(); + + if (auto argument_column_array = typeid_cast(array_column.get())) + source = GatherUtils::createArraySource(*argument_column_array, false, size); + else + throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; + + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); + + if (pop_front) + GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, 1); + else + GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, -1); + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + +private: + bool pop_front; + const char * name; +}; + +} diff --git a/dbms/src/Functions/arrayPopBack.cpp b/dbms/src/Functions/arrayPopBack.cpp new file mode 100644 index 00000000000..411333b8a14 --- /dev/null +++ b/dbms/src/Functions/arrayPopBack.cpp @@ -0,0 +1,21 @@ +#include +#include + + +namespace DB +{ + +class FunctionArrayPopBack : public FunctionArrayPop +{ +public: + static constexpr auto name = "arrayPopBack"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + FunctionArrayPopBack() : FunctionArrayPop(false, name) {} +}; + +void registerFunctionArrayPopBack(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayPopFront.cpp b/dbms/src/Functions/arrayPopFront.cpp new file mode 100644 index 00000000000..59adfc76cc0 --- /dev/null +++ b/dbms/src/Functions/arrayPopFront.cpp @@ -0,0 +1,21 @@ +#include +#include + + +namespace DB +{ + +class FunctionArrayPopFront : public FunctionArrayPop +{ +public: + static constexpr auto name = "arrayPopFront"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + FunctionArrayPopFront() : FunctionArrayPop(true, name) {} +}; + +void registerFunctionArrayPopFront(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayPush.h b/dbms/src/Functions/arrayPush.h new file mode 100644 index 00000000000..4d06571ea71 --- /dev/null +++ b/dbms/src/Functions/arrayPush.h @@ -0,0 +1,114 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionArrayPush : public IFunction +{ +public: + FunctionArrayPush(const Context & context, bool push_front, const char * name) + : context(context), push_front(push_front), name(name) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments[0]->onlyNull()) + return arguments[0]; + + auto array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array but it has type " + + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto nested_type = array_type->getNestedType(); + + DataTypes types = {nested_type, arguments[1]}; + + return std::make_shared(getLeastSupertype(types)); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & return_type = block.getByPosition(result).type; + + if (return_type->onlyNull()) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto result_column = return_type->createColumn(); + + auto array_column = block.getByPosition(arguments[0]).column; + auto appended_column = block.getByPosition(arguments[1]).column; + + if (!block.getByPosition(arguments[0]).type->equals(*return_type)) + array_column = castColumn(block.getByPosition(arguments[0]), return_type, context); + + const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); + if (!block.getByPosition(arguments[1]).type->equals(*return_nested_type)) + appended_column = castColumn(block.getByPosition(arguments[1]), return_nested_type, context); + + std::unique_ptr array_source; + std::unique_ptr value_source; + + size_t size = array_column->size(); + bool is_const = false; + + if (auto const_array_column = typeid_cast(array_column.get())) + { + is_const = true; + array_column = const_array_column->getDataColumnPtr(); + } + + if (auto argument_column_array = typeid_cast(array_column.get())) + array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); + else + throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; + + + bool is_appended_const = false; + if (auto const_appended_column = typeid_cast(appended_column.get())) + { + is_appended_const = true; + appended_column = const_appended_column->getDataColumnPtr(); + } + + value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); + + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); + + GatherUtils::push(*array_source, *value_source, *sink, push_front); + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + +private: + const Context & context; + bool push_front; + const char * name; +}; + +} diff --git a/dbms/src/Functions/arrayPushBack.cpp b/dbms/src/Functions/arrayPushBack.cpp new file mode 100644 index 00000000000..f1b57199ed9 --- /dev/null +++ b/dbms/src/Functions/arrayPushBack.cpp @@ -0,0 +1,21 @@ +#include +#include + + +namespace DB +{ + +class FunctionArrayPushBack : public FunctionArrayPush +{ +public: + static constexpr auto name = "arrayPushBack"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayPushBack(const Context & context) : FunctionArrayPush(context, false, name) {} +}; + +void registerFunctionArrayPushBack(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayPushFront.cpp b/dbms/src/Functions/arrayPushFront.cpp new file mode 100644 index 00000000000..484681e13ba --- /dev/null +++ b/dbms/src/Functions/arrayPushFront.cpp @@ -0,0 +1,23 @@ +#include +#include + + +namespace DB +{ + + +class FunctionArrayPushFront : public FunctionArrayPush +{ +public: + static constexpr auto name = "arrayPushFront"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayPushFront(const Context & context) : FunctionArrayPush(context, true, name) {} +}; + + +void registerFunctionArrayPushFront(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayReduce.cpp b/dbms/src/Functions/arrayReduce.cpp new file mode 100644 index 00000000000..31aa4d49f18 --- /dev/null +++ b/dbms/src/Functions/arrayReduce.cpp @@ -0,0 +1,195 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** Applies an aggregate function to array and returns its result. + * If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size. + * + * arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...` + * If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function. + */ +class FunctionArrayReduce : public IFunction +{ +public: + static constexpr auto name = "arrayReduce"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + 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; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + /// lazy initialization in getReturnTypeImpl + /// TODO: init in FunctionBuilder + mutable AggregateFunctionPtr aggregate_function; +}; + + +DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + /// The first argument is a constant string with the name of the aggregate function + /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). + + if (arguments.size() < 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst(arguments[0].column.get()); + if (!aggregate_function_name_column) + throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypes argument_types(arguments.size() - 1); + for (size_t i = 1, size = arguments.size(); i < size; ++i) + { + const DataTypeArray * arg = checkAndGetDataType(arguments[i].type.get()); + if (!arg) + throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type " + + arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + argument_types[i - 1] = arg->getNestedType(); + } + + if (!aggregate_function) + { + String aggregate_function_name_with_params = aggregate_function_name_column->getValue(); + + if (aggregate_function_name_with_params.empty()) + throw Exception("First argument for function " + getName() + " (name of aggregate function) cannot be empty.", + ErrorCodes::BAD_ARGUMENTS); + + String aggregate_function_name; + Array params_row; + getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params, + aggregate_function_name, params_row, "function " + getName()); + + aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row); + } + + return aggregate_function->getReturnType(); +} + + +void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + IAggregateFunction & agg_func = *aggregate_function.get(); + AlignedBuffer place_holder(agg_func.sizeOfData(), agg_func.alignOfData()); + AggregateDataPtr place = place_holder.data(); + + std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; + + size_t rows = input_rows_count; + + /// Aggregate functions do not support constant columns. Therefore, we materialize them. + std::vector materialized_columns; + + const size_t num_arguments_columns = arguments.size() - 1; + + std::vector aggregate_arguments_vec(num_arguments_columns); + const ColumnArray::Offsets * offsets = nullptr; + + for (size_t i = 0; i < num_arguments_columns; ++i) + { + const IColumn * col = block.getByPosition(arguments[i + 1]).column.get(); + const ColumnArray::Offsets * offsets_i = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(col)) + { + aggregate_arguments_vec[i] = &arr->getData(); + offsets_i = &arr->getOffsets(); + } + else if (const ColumnConst * const_arr = checkAndGetColumnConst(col)) + { + materialized_columns.emplace_back(const_arr->convertToFullColumn()); + const auto & arr = typeid_cast(*materialized_columns.back().get()); + aggregate_arguments_vec[i] = &arr.getData(); + offsets_i = &arr.getOffsets(); + } + else + throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + if (i == 0) + offsets = offsets_i; + else if (*offsets_i != *offsets) + throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + const IColumn ** aggregate_arguments = aggregate_arguments_vec.data(); + + MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn(); + IColumn & res_col = *result_holder; + + /// AggregateFunction's states should be inserted into column using specific way + auto res_col_aggregate_function = typeid_cast(&res_col); + + if (!res_col_aggregate_function && agg_func.isState()) + throw Exception("State function " + agg_func.getName() + " inserts results into non-state column " + + block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + agg_func.create(place); + ColumnArray::Offset next_offset = (*offsets)[i]; + + try + { + for (size_t j = current_offset; j < next_offset; ++j) + agg_func.add(place, aggregate_arguments, j, arena.get()); + + if (!res_col_aggregate_function) + agg_func.insertResultInto(place, res_col); + else + res_col_aggregate_function->insertFrom(place); + } + catch (...) + { + agg_func.destroy(place); + throw; + } + + agg_func.destroy(place); + current_offset = next_offset; + } + + block.getByPosition(result).column = std::move(result_holder); +} + + +void registerFunctionArrayReduce(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayResize.cpp b/dbms/src/Functions/arrayResize.cpp new file mode 100644 index 00000000000..935e2039fe3 --- /dev/null +++ b/dbms/src/Functions/arrayResize.cpp @@ -0,0 +1,148 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionArrayResize : public IFunction +{ +public: + static constexpr auto name = "arrayResize"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayResize(const Context & context) : context(context) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments[0]->onlyNull()) + return arguments[0]; + + auto array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array but it has type " + + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (WhichDataType(array_type->getNestedType()).isNothing()) + throw Exception("Function " + getName() + " cannot resize " + array_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isInteger(removeNullable(arguments[1])) && !arguments[1]->onlyNull()) + throw Exception( + "Argument " + toString(1) + " for function " + getName() + " must be integer but it has type " + + arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (number_of_arguments == 2) + return arguments[0]; + else /* if (number_of_arguments == 3) */ + return std::make_shared(getLeastSupertype({array_type->getNestedType(), arguments[2]})); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & return_type = block.getByPosition(result).type; + + if (return_type->onlyNull()) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto result_column = return_type->createColumn(); + + auto array_column = block.getByPosition(arguments[0]).column; + auto size_column = block.getByPosition(arguments[1]).column; + + if (!block.getByPosition(arguments[0]).type->equals(*return_type)) + array_column = castColumn(block.getByPosition(arguments[0]), return_type, context); + + const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); + size_t size = array_column->size(); + + ColumnPtr appended_column; + if (arguments.size() == 3) + { + appended_column = block.getByPosition(arguments[2]).column; + if (!block.getByPosition(arguments[2]).type->equals(*return_nested_type)) + appended_column = castColumn(block.getByPosition(arguments[2]), return_nested_type, context); + } + else + appended_column = return_nested_type->createColumnConstWithDefaultValue(size); + + std::unique_ptr array_source; + std::unique_ptr value_source; + + bool is_const = false; + + if (auto const_array_column = typeid_cast(array_column.get())) + { + is_const = true; + array_column = const_array_column->getDataColumnPtr(); + } + + if (auto argument_column_array = typeid_cast(array_column.get())) + array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); + else + throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; + + + bool is_appended_const = false; + if (auto const_appended_column = typeid_cast(appended_column.get())) + { + is_appended_const = true; + appended_column = const_appended_column->getDataColumnPtr(); + } + + value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); + + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); + + if (size_column->isColumnConst()) + GatherUtils::resizeConstantSize(*array_source, *value_source, *sink, size_column->getInt(0)); + else + GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column); + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + +private: + const Context & context; +}; + + +void registerFunctionArrayResize(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp new file mode 100644 index 00000000000..1d4ca49d61a --- /dev/null +++ b/dbms/src/Functions/arrayReverse.cpp @@ -0,0 +1,365 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// TODO Add generic implementation. + +class FunctionArrayReverse : public IFunction +{ +public: + static constexpr auto name = "arrayReverse"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception("Argument for function " + getName() + " must be array.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, + size_t input_rows_count); + + template + bool executeNumber( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col); + + bool executeFixedString( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col); + + bool executeString( + const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col); +}; + + +void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + if (executeConst(block, arguments, result, input_rows_count)) + return; + + const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + if (!array) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + auto res_ptr = array->cloneEmpty(); + ColumnArray & res = static_cast(*res_ptr); + res.getOffsetsPtr() = array->getOffsetsPtr(); + + const IColumn & src_data = array->getData(); + const ColumnArray::Offsets & offsets = array->getOffsets(); + IColumn & res_data = res.getData(); + + const ColumnNullable * nullable_col = nullptr; + ColumnNullable * nullable_res_col = nullptr; + + const IColumn * inner_col; + IColumn * inner_res_col; + + if (src_data.isColumnNullable()) + { + nullable_col = static_cast(&src_data); + inner_col = &nullable_col->getNestedColumn(); + + nullable_res_col = static_cast(&res_data); + inner_res_col = &nullable_res_col->getNestedColumn(); + } + else + { + inner_col = &src_data; + inner_res_col = &res_data; + } + + if (!( executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + || executeFixedString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + block.getByPosition(result).column = std::move(res_ptr); +} + +bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, + size_t input_rows_count) +{ + if (const ColumnConst * const_array = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) + { + Array arr = const_array->getValue(); + + size_t size = arr.size(); + Array res(size); + + for (size_t i = 0; i < size; ++i) + res[i] = arr[size - i - 1]; + + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, res); + + return true; + } + else + return false; +} + +template +bool FunctionArrayReverse::executeNumber( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col) +{ + auto do_reverse = [](const auto & src_data, const auto & src_offsets, auto & res_data) + { + size_t size = src_offsets.size(); + ColumnArray::Offset src_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const auto * src = &src_data[src_prev_offset]; + const auto * src_end = &src_data[src_offsets[i]]; + + if (src == src_end) + continue; + + auto dst = &res_data[src_offsets[i] - 1]; + + while (src < src_end) + { + *dst = *src; + ++src; + --dst; + } + + src_prev_offset = src_offsets[i]; + } + }; + + if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) + { + const PaddedPODArray & src_data = src_data_concrete->getData(); + PaddedPODArray & res_data = typeid_cast &>(res_data_col).getData(); + res_data.resize(src_data.size()); + do_reverse(src_data, src_offsets, res_data); + + if ((nullable_col) && (nullable_res_col)) + { + /// Make a reverted null map. + const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); + auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + res_null_map.resize(src_data.size()); + do_reverse(src_null_map, src_offsets, res_null_map); + } + + return true; + } + else + return false; +} + +bool FunctionArrayReverse::executeFixedString( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col) +{ + if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) + { + const size_t n = src_data_concrete->getN(); + const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); + ColumnFixedString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + size_t size = src_offsets.size(); + res_data.resize(src_data.size()); + + ColumnArray::Offset src_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const UInt8 * src = &src_data[src_prev_offset * n]; + const UInt8 * src_end = &src_data[src_offsets[i] * n]; + + if (src == src_end) + continue; + + UInt8 * dst = &res_data[src_offsets[i] * n - n]; + + while (src < src_end) + { + /// NOTE: memcpySmallAllowReadWriteOverflow15 doesn't work correctly here. + memcpy(dst, src, n); + src += n; + dst -= n; + } + + src_prev_offset = src_offsets[i]; + } + + if ((nullable_col) && (nullable_res_col)) + { + /// Make a reverted null map. + const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); + auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + res_null_map.resize(src_null_map.size()); + + ColumnArray::Offset src_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const UInt8 * src = &src_null_map[src_prev_offset]; + const UInt8 * src_end = &src_null_map[src_offsets[i]]; + + if (src == src_end) + continue; + + UInt8 * dst = &res_null_map[src_offsets[i] - 1]; + + while (src < src_end) + { + *dst = *src; + ++src; + --dst; + } + + src_prev_offset = src_offsets[i]; + } + } + + return true; + } + else + return false; +} + +bool FunctionArrayReverse::executeString( + const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, + IColumn & res_data_col, + const ColumnNullable * nullable_col, + ColumnNullable * nullable_res_col) +{ + if (const ColumnString * src_data_concrete = checkAndGetColumn(&src_data)) + { + const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); + ColumnString::Offsets & res_string_offsets = typeid_cast(res_data_col).getOffsets(); + + const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); + ColumnString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + + size_t size = src_array_offsets.size(); + res_string_offsets.resize(src_string_offsets.size()); + res_data.resize(src_data.size()); + + ColumnArray::Offset src_array_prev_offset = 0; + ColumnString::Offset res_string_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_array_offsets[i] != src_array_prev_offset) + { + size_t array_size = src_array_offsets[i] - src_array_prev_offset; + + for (size_t j = 0; j < array_size; ++j) + { + size_t j_reversed = array_size - j - 1; + + auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1]; + size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos; + + memcpySmallAllowReadWriteOverflow15(&res_data[res_string_prev_offset], &src_data[src_pos], string_size); + + res_string_prev_offset += string_size; + res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset; + } + } + + src_array_prev_offset = src_array_offsets[i]; + } + + if ((nullable_col) && (nullable_res_col)) + { + /// Make a reverted null map. + const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); + auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + res_null_map.resize(src_string_offsets.size()); + + size_t size = src_string_offsets.size(); + ColumnArray::Offset src_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const auto * src = &src_null_map[src_prev_offset]; + const auto * src_end = &src_null_map[src_array_offsets[i]]; + + if (src == src_end) + continue; + + auto dst = &res_null_map[src_array_offsets[i] - 1]; + + while (src < src_end) + { + *dst = *src; + ++src; + --dst; + } + + src_prev_offset = src_array_offsets[i]; + } + } + + return true; + } + else + return false; +} + + +void registerFunctionArrayReverse(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/arraySlice.cpp b/dbms/src/Functions/arraySlice.cpp new file mode 100644 index 00000000000..08854ee1079 --- /dev/null +++ b/dbms/src/Functions/arraySlice.cpp @@ -0,0 +1,165 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** arraySlice(arr, offset, length) - make slice of array. Offsets and length may be < 0 or Null + * - if offset < 0, indexation from right element + * - if length < 0, length = len(array) - (positive_index(offset) - 1) + length + * indexation: + * [ 1, 2, 3, 4, 5, 6] + * [-6, -5, -4, -3, -2, -1] + * examples: + * arraySlice([1, 2, 3, 4, 5, 6], -4, 2) -> [3, 4] + * arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] (6 - (2 - 1) + (-1) = 4) + * arraySlice([1, 2, 3, 4, 5, 6], -5, -1) = arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] + */ +class FunctionArraySlice : public IFunction +{ +public: + static constexpr auto name = "arraySlice"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments[0]->onlyNull()) + return arguments[0]; + + auto array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array but it has type " + + arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + for (size_t i = 1; i < number_of_arguments; ++i) + { + if (!isInteger(removeNullable(arguments[i])) && !arguments[i]->onlyNull()) + throw Exception( + "Argument " + toString(i) + " for function " + getName() + " must be integer but it has type " + + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & return_type = block.getByPosition(result).type; + + if (return_type->onlyNull()) + { + block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto result_column = return_type->createColumn(); + + auto & array_column = block.getByPosition(arguments[0]).column; + const auto & offset_column = block.getByPosition(arguments[1]).column; + const auto & length_column = arguments.size() > 2 ? block.getByPosition(arguments[2]).column : nullptr; + + std::unique_ptr source; + + size_t size = array_column->size(); + bool is_const = false; + + if (auto const_array_column = typeid_cast(array_column.get())) + { + is_const = true; + array_column = const_array_column->getDataColumnPtr(); + } + + if (auto argument_column_array = typeid_cast(array_column.get())) + source = GatherUtils::createArraySource(*argument_column_array, is_const, size); + else + throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR}; + + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); + + if (offset_column->onlyNull()) + { + if (!length_column || length_column->onlyNull()) + { + block.getByPosition(result).column = array_column; + return; + } + else if (length_column->isColumnConst()) + GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, length_column->getInt(0)); + else + { + auto const_offset_column = ColumnConst::create(ColumnInt8::create(1, 1), size); + GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *const_offset_column, *length_column); + } + } + else if (offset_column->isColumnConst()) + { + ssize_t offset = offset_column->getUInt(0); + + if (!length_column || length_column->onlyNull()) + { + if (offset > 0) + GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, static_cast(offset - 1)); + else + GatherUtils::sliceFromRightConstantOffsetUnbounded(*source, *sink, static_cast(-offset)); + } + else if (length_column->isColumnConst()) + { + ssize_t length = length_column->getInt(0); + if (offset > 0) + GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, static_cast(offset - 1), length); + else + GatherUtils::sliceFromRightConstantOffsetBounded(*source, *sink, static_cast(-offset), length); + } + else + GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column); + } + else + { + if (!length_column || length_column->onlyNull()) + GatherUtils::sliceDynamicOffsetUnbounded(*source, *sink, *offset_column); + else + GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column); + } + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } +}; + + +void registerFunctionArraySlice(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} diff --git a/dbms/src/Functions/arrayUniq.cpp b/dbms/src/Functions/arrayUniq.cpp new file mode 100644 index 00000000000..4c567c3a9d1 --- /dev/null +++ b/dbms/src/Functions/arrayUniq.cpp @@ -0,0 +1,367 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/// Counts the number of different elements in the array, or the number of different tuples from the elements at the corresponding positions in several arrays. +/// NOTE The implementation partially matches arrayEnumerateUniq. +class FunctionArrayUniq : public IFunction +{ +public: + static constexpr auto name = "arrayUniq"; + + static FunctionPtr create(const Context &) { return std::make_shared(); } + + 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 + { + if (arguments.size() == 0) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < arguments.size(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception("All arguments for function " + getName() + " must be arrays but argument " + + toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. + static constexpr size_t INITIAL_SIZE_DEGREE = 9; + + template + bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + + bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + + bool execute128bit( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const ColumnRawPtrs & null_maps, + ColumnUInt32::Container & res_values, + bool has_nullable_columns); + + void executeHashed( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values); +}; + + +void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) +{ + Columns array_columns(arguments.size()); + const ColumnArray::Offsets * offsets = nullptr; + ColumnRawPtrs data_columns(arguments.size()); + ColumnRawPtrs original_data_columns(arguments.size()); + ColumnRawPtrs null_maps(arguments.size()); + + bool has_nullable_columns = false; + + for (size_t i = 0; i < arguments.size(); ++i) + { + ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; + const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + if (!array) + { + const ColumnConst * const_array = checkAndGetColumnConst( + block.getByPosition(arguments[i]).column.get()); + if (!const_array) + throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() + + " of " + toString(i + 1) + getOrdinalSuffix(i + 1) + " argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + array_ptr = const_array->convertToFullColumn(); + array = static_cast(array_ptr.get()); + } + + array_columns[i] = array_ptr; + + const ColumnArray::Offsets & offsets_i = array->getOffsets(); + if (i == 0) + offsets = &offsets_i; + else if (offsets_i != *offsets) + throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + + data_columns[i] = &array->getData(); + original_data_columns[i] = data_columns[i]; + + if (data_columns[i]->isColumnNullable()) + { + has_nullable_columns = true; + const auto & nullable_col = static_cast(*data_columns[i]); + data_columns[i] = &nullable_col.getNestedColumn(); + null_maps[i] = &nullable_col.getNullMapColumn(); + } + else + null_maps[i] = nullptr; + } + + const ColumnArray * first_array = static_cast(array_columns[0].get()); + const IColumn * first_null_map = null_maps[0]; + auto res = ColumnUInt32::create(); + + ColumnUInt32::Container & res_values = res->getData(); + res_values.resize(offsets->size()); + + if (arguments.size() == 1) + { + if (!( executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeNumber(first_array, first_null_map, res_values) + || executeString(first_array, first_null_map, res_values))) + executeHashed(*offsets, original_data_columns, res_values); + } + else + { + if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) + executeHashed(*offsets, original_data_columns, res_values); + } + + block.getByPosition(result).column = std::move(res); +} + +template +bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +{ + const IColumn * inner_col; + + const auto & array_data = array->getData(); + if (array_data.isColumnNullable()) + { + const auto & nullable_col = static_cast(array_data); + inner_col = &nullable_col.getNestedColumn(); + } + else + inner_col = &array_data; + + const ColumnVector * nested = checkAndGetColumn>(inner_col); + if (!nested) + return false; + const ColumnArray::Offsets & offsets = array->getOffsets(); + const typename ColumnVector::Container & values = nested->getData(); + + using Set = ClearableHashSet, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + + const PaddedPODArray * null_map_data = nullptr; + if (null_map) + null_map_data = &static_cast(null_map)->getData(); + + Set set; + ColumnArray::Offset prev_off = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + set.clear(); + bool found_null = false; + ColumnArray::Offset off = offsets[i]; + for (ColumnArray::Offset j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + found_null = true; + else + set.insert(values[j]); + } + + res_values[i] = set.size() + found_null; + prev_off = off; + } + return true; +} + +bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +{ + const IColumn * inner_col; + + const auto & array_data = array->getData(); + if (array_data.isColumnNullable()) + { + const auto & nullable_col = static_cast(array_data); + inner_col = &nullable_col.getNestedColumn(); + } + else + inner_col = &array_data; + + const ColumnString * nested = checkAndGetColumn(inner_col); + if (!nested) + return false; + const ColumnArray::Offsets & offsets = array->getOffsets(); + + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + + const PaddedPODArray * null_map_data = nullptr; + if (null_map) + null_map_data = &static_cast(null_map)->getData(); + + Set set; + ColumnArray::Offset prev_off = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + set.clear(); + bool found_null = false; + ColumnArray::Offset off = offsets[i]; + for (ColumnArray::Offset j = prev_off; j < off; ++j) + { + if (null_map_data && ((*null_map_data)[j] == 1)) + found_null = true; + else + set.insert(nested->getDataAt(j)); + } + + res_values[i] = set.size() + found_null; + prev_off = off; + } + return true; +} + + +bool FunctionArrayUniq::execute128bit( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const ColumnRawPtrs & null_maps, + ColumnUInt32::Container & res_values, + bool has_nullable_columns) +{ + size_t count = columns.size(); + size_t keys_bytes = 0; + Sizes key_sizes(count); + + for (size_t j = 0; j < count; ++j) + { + if (!columns[j]->isFixedAndContiguous()) + return false; + key_sizes[j] = columns[j]->sizeOfValueIfFixed(); + keys_bytes += key_sizes[j]; + } + if (has_nullable_columns) + keys_bytes += std::tuple_size>::value; + + if (keys_bytes > 16) + return false; + + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + + /// Suppose that, for a given row, each of the N columns has an array whose length is M. + /// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed: + /// + /// col1 ... colN + /// + /// arr_1[1], ..., arr_N[1] -> pack into a binary blob b1 + /// . + /// . + /// . + /// arr_1[M], ..., arr_N[M] -> pack into a binary blob bM + /// + /// Each binary blob is inserted into a hash table. + /// + Set set; + ColumnArray::Offset prev_off = 0; + for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) + { + set.clear(); + ColumnArray::Offset off = offsets[i]; + for (ColumnArray::Offset j = prev_off; j < off; ++j) + { + if (has_nullable_columns) + { + KeysNullMap bitmap{}; + + for (ColumnArray::Offset i = 0; i < columns.size(); ++i) + { + if (null_maps[i]) + { + const auto & null_map = static_cast(*null_maps[i]).getData(); + if (null_map[j] == 1) + { + ColumnArray::Offset bucket = i / 8; + ColumnArray::Offset offset = i % 8; + bitmap[bucket] |= UInt8(1) << offset; + } + } + } + set.insert(packFixed(j, count, columns, key_sizes, bitmap)); + } + else + set.insert(packFixed(j, count, columns, key_sizes)); + } + + res_values[i] = set.size(); + prev_off = off; + } + + return true; +} + +void FunctionArrayUniq::executeHashed( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values) +{ + size_t count = columns.size(); + + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + + Set set; + ColumnArray::Offset prev_off = 0; + for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) + { + set.clear(); + ColumnArray::Offset off = offsets[i]; + for (ColumnArray::Offset j = prev_off; j < off; ++j) + set.insert(hash128(j, count, columns)); + + res_values[i] = set.size(); + prev_off = off; + } +} + + +void registerFunctionArrayUniq(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/assumeNotNull.cpp b/dbms/src/Functions/assumeNotNull.cpp new file mode 100644 index 00000000000..509cc84969f --- /dev/null +++ b/dbms/src/Functions/assumeNotNull.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function assumeNotNull which takes 1 argument and works as follows: +/// - if the argument is a nullable column, return its embedded column; +/// - otherwise return the original argument. +/// NOTE: assumeNotNull may not be called with the NULL value. +class FunctionAssumeNotNull : public IFunction +{ +public: + static constexpr auto name = "assumeNotNull"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return removeNullable(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + const ColumnPtr & col = block.getByPosition(arguments[0]).column; + ColumnPtr & res_col = block.getByPosition(result).column; + + if (col->isColumnNullable()) + { + const ColumnNullable & nullable_col = static_cast(*col); + res_col = nullable_col.getNestedColumnPtr(); + } + else + res_col = col; + } +}; + + +void registerFunctionAssumeNotNull(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/bar.cpp b/dbms/src/Functions/bar.cpp new file mode 100644 index 00000000000..6cb40154ad7 --- /dev/null +++ b/dbms/src/Functions/bar.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max. + * Returns a string with nice Unicode-art bar with resolution of 1/8 part of symbol. + */ +class FunctionBar : public IFunction +{ +public: + static constexpr auto name = "bar"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 3 && arguments.size() != 4) + throw Exception("Function " + getName() + + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed " + + toString(arguments.size()) + + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isNumber(arguments[0]) || !isNumber(arguments[1]) || !isNumber(arguments[2]) + || (arguments.size() == 4 && !isNumber(arguments[3]))) + throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + Int64 min = extractConstant(block, arguments, 1, "Second"); /// The level at which the line has zero length. + Int64 max = extractConstant(block, arguments, 2, "Third"); /// The level at which the line has the maximum length. + + /// The maximum width of the bar in characters, by default. + Float64 max_width = arguments.size() == 4 ? extractConstant(block, arguments, 3, "Fourth") : 80; + + if (max_width < 1) + throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (max_width > 1000) + throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + const auto & src = *block.getByPosition(arguments[0]).column; + + auto res_column = ColumnString::create(); + + if (executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width)) + { + block.getByPosition(result).column = std::move(res_column); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + template + T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const + { + const auto & column = *block.getByPosition(arguments[argument_pos]).column; + + if (!column.isColumnConst()) + throw Exception( + which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); + + return applyVisitor(FieldVisitorConvertToNumber(), column[0]); + } + + template + static void fill(const PaddedPODArray & src, + ColumnString::Chars_t & dst_chars, + ColumnString::Offsets & dst_offsets, + Int64 min, + Int64 max, + Float64 max_width) + { + size_t size = src.size(); + size_t current_offset = 0; + + dst_offsets.resize(size); + dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated. + + for (size_t i = 0; i < size; ++i) + { + Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width); + size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; + dst_chars.resize(next_size); + UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset])); + current_offset = next_size; + dst_offsets[i] = current_offset; + } + } + + template + static void fill(T src, String & dst_chars, Int64 min, Int64 max, Float64 max_width) + { + Float64 width = UnicodeBar::getWidth(src, min, max, max_width); + dst_chars.resize(UnicodeBar::getWidthInBytes(width)); + UnicodeBar::render(width, dst_chars.data()); + } + + template + static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width) + { + if (const ColumnVector * col = checkAndGetColumn>(&src)) + { + fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width); + return true; + } + else + return false; + } +}; + + +void registerFunctionBar(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/blockNumber.cpp b/dbms/src/Functions/blockNumber.cpp new file mode 100644 index 00000000000..f6acb682318 --- /dev/null +++ b/dbms/src/Functions/blockNumber.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Incremental block number among calls of this function. */ +class FunctionBlockNumber : public IFunction +{ +private: + std::atomic block_number{0}; + +public: + static constexpr auto name = "blockNumber"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + size_t current_block_number = block_number++; + block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number); + } +}; + + +void registerFunctionBlockNumber(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/blockSize.cpp b/dbms/src/Functions/blockSize.cpp new file mode 100644 index 00000000000..4b4c18edbf9 --- /dev/null +++ b/dbms/src/Functions/blockSize.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** blockSize() - get the block size in number of rows. + */ +class FunctionBlockSize : public IFunction +{ +public: + static constexpr auto name = "blockSize"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count); + } +}; + + +void registerFunctionBlockSize(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/coalesce.cpp b/dbms/src/Functions/coalesce.cpp new file mode 100644 index 00000000000..cb65a019a31 --- /dev/null +++ b/dbms/src/Functions/coalesce.cpp @@ -0,0 +1,169 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function coalesce which takes a set of arguments and +/// returns the value of the leftmost non-null argument. If no such value is +/// found, coalesce() returns NULL. +class FunctionCoalesce : public IFunction +{ +public: + static constexpr auto name = "coalesce"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + FunctionCoalesce(const Context & context) : context(context) {} + + std::string getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + /// Skip all NULL arguments. If any argument is non-Nullable, skip all next arguments. + DataTypes filtered_args; + filtered_args.reserve(arguments.size()); + for (const auto & arg : arguments) + { + if (arg->onlyNull()) + continue; + + filtered_args.push_back(arg); + + if (!arg->isNullable()) + break; + } + + DataTypes new_args; + for (size_t i = 0; i < filtered_args.size(); ++i) + { + bool is_last = i + 1 == filtered_args.size(); + + if (is_last) + new_args.push_back(filtered_args[i]); + else + new_args.push_back(removeNullable(filtered_args[i])); + } + + if (new_args.empty()) + return std::make_shared(std::make_shared()); + if (new_args.size() == 1) + return new_args.front(); + + auto res = getLeastSupertype(new_args); + + /// if last argument is not nullable, result should be also not nullable + if (!new_args.back()->isNullable() && res->isNullable()) + res = removeNullable(res); + + return res; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + /// coalesce(arg0, arg1, ..., argN) is essentially + /// multiIf(isNotNull(arg0), assumeNotNull(arg0), isNotNull(arg1), assumeNotNull(arg1), ..., argN) + /// with constant NULL arguments removed. + + ColumnNumbers filtered_args; + filtered_args.reserve(arguments.size()); + for (const auto & arg : arguments) + { + const auto & type = block.getByPosition(arg).type; + + if (type->onlyNull()) + continue; + + filtered_args.push_back(arg); + + if (!type->isNullable()) + break; + } + + auto is_not_null = FunctionFactory::instance().get("isNotNull", context); + auto assume_not_null = FunctionFactory::instance().get("assumeNotNull", context); + auto multi_if = FunctionFactory::instance().get("multiIf", context); + + ColumnNumbers multi_if_args; + + Block temp_block = block; + + for (size_t i = 0; i < filtered_args.size(); ++i) + { + size_t res_pos = temp_block.columns(); + bool is_last = i + 1 == filtered_args.size(); + + if (is_last) + { + multi_if_args.push_back(filtered_args[i]); + } + else + { + temp_block.insert({nullptr, std::make_shared(), ""}); + is_not_null->build({temp_block.getByPosition(filtered_args[i])})->execute(temp_block, {filtered_args[i]}, res_pos, input_rows_count); + temp_block.insert({nullptr, removeNullable(block.getByPosition(filtered_args[i]).type), ""}); + assume_not_null->build({temp_block.getByPosition(filtered_args[i])})->execute(temp_block, {filtered_args[i]}, res_pos + 1, input_rows_count); + + multi_if_args.push_back(res_pos); + multi_if_args.push_back(res_pos + 1); + } + } + + /// If all arguments appeared to be NULL. + if (multi_if_args.empty()) + { + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + if (multi_if_args.size() == 1) + { + block.getByPosition(result).column = block.getByPosition(multi_if_args.front()).column; + return; + } + + ColumnsWithTypeAndName multi_if_args_elems; + multi_if_args_elems.reserve(multi_if_args.size()); + for (auto column_num : multi_if_args) + multi_if_args_elems.emplace_back(temp_block.getByPosition(column_num)); + + multi_if->build(multi_if_args_elems)->execute(temp_block, multi_if_args, result, input_rows_count); + + ColumnPtr res = std::move(temp_block.getByPosition(result).column); + + /// if last argument is not nullable, result should be also not nullable + if (!block.getByPosition(multi_if_args.back()).column->isColumnNullable() && res->isColumnNullable()) + res = static_cast(*res).getNestedColumnPtr(); + + block.getByPosition(result).column = std::move(res); + } + +private: + const Context & context; +}; + + +void registerFunctionCoalesce(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/concat.cpp b/dbms/src/Functions/concat.cpp new file mode 100644 index 00000000000..67bf020e340 --- /dev/null +++ b/dbms/src/Functions/concat.cpp @@ -0,0 +1,193 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +using namespace GatherUtils; + + +template +class ConcatImpl : public IFunction +{ +public: + static constexpr auto name = Name::name; + ConcatImpl(const Context & context) : context(context) {} + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override + { + return true; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isInjective(const Block &) override + { + return is_injective; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be at least 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (const auto arg_idx : ext::range(0, arguments.size())) + { + const auto arg = arguments[arg_idx].get(); + if (!isStringOrFixedString(arg)) + throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + if (arguments.size() == 2) + executeBinary(block, arguments, result, input_rows_count); + else + executeNAry(block, arguments, result, input_rows_count); + } + +private: + const Context & context; + + void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); + const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); + + const ColumnString * c0_string = checkAndGetColumn(c0); + const ColumnString * c1_string = checkAndGetColumn(c1); + const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); + const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); + + auto c_res = ColumnString::create(); + + if (c0_string && c1_string) + concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); + else if (c0_string && c1_const_string) + concat(StringSource(*c0_string), ConstSource(*c1_const_string), StringSink(*c_res, c0->size())); + else if (c0_const_string && c1_string) + concat(ConstSource(*c0_const_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); + else + { + /// Fallback: use generic implementation for not very important cases. + executeNAry(block, arguments, result, input_rows_count); + return; + } + + block.getByPosition(result).column = std::move(c_res); + } + + void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + size_t num_sources = arguments.size(); + StringSources sources(num_sources); + + for (size_t i = 0; i < num_sources; ++i) + sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column); + + auto c_res = ColumnString::create(); + concat(sources, StringSink(*c_res, input_rows_count)); + block.getByPosition(result).column = std::move(c_res); + } +}; + + +struct NameConcat +{ + static constexpr auto name = "concat"; +}; +struct NameConcatAssumeInjective +{ + static constexpr auto name = "concatAssumeInjective"; +}; + +using FunctionConcat = ConcatImpl; +using FunctionConcatAssumeInjective = ConcatImpl; + + +/// Also works with arrays. +class FunctionBuilderConcat : public FunctionBuilderImpl +{ +public: + static constexpr auto name = "concat"; + static FunctionBuilderPtr create(const Context & context) { return std::make_shared(context); } + + FunctionBuilderConcat(const Context & context) : context(context) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + +protected: + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (isArray(arguments.at(0).type)) + return FunctionFactory::instance().get("arrayConcat", context)->build(arguments); + else + return std::make_shared( + FunctionConcat::create(context), + ext::map(arguments, [](const auto & elem) { return elem.type; }), + return_type); + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be at least 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return getLeastSupertype(arguments); + } + +private: + const Context & context; +}; + + +void registerFunctionsConcat(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/FunctionsCharset.cpp b/dbms/src/Functions/convertCharset.cpp similarity index 97% rename from dbms/src/Functions/FunctionsCharset.cpp rename to dbms/src/Functions/convertCharset.cpp index db2064a2cfc..e4097b8a6f3 100644 --- a/dbms/src/Functions/FunctionsCharset.cpp +++ b/dbms/src/Functions/convertCharset.cpp @@ -171,7 +171,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { for (size_t i : ext::range(0, 3)) - if (!arguments[i]->isString()) + if (!isString(arguments[i])) throw Exception("Illegal type " + arguments[i]->getName() + " of argument of function " + getName() + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -210,20 +210,11 @@ public: }; -void registerFunctionsCharset(FunctionFactory & factory) +void registerFunctionConvertCharset(FunctionFactory & factory) { factory.registerFunction(); } } -#else - -namespace DB -{ - class FunctionFactory; - void registerFunctionsCharset(FunctionFactory & factory) { (void)factory; } -} - - #endif diff --git a/dbms/src/Functions/countEqual.cpp b/dbms/src/Functions/countEqual.cpp new file mode 100644 index 00000000000..08f00422560 --- /dev/null +++ b/dbms/src/Functions/countEqual.cpp @@ -0,0 +1,18 @@ +#include +#include + + +namespace DB +{ + +struct NameCountEqual { static constexpr auto name = "countEqual"; }; + +using FunctionCountEqual = FunctionArrayIndex; + +void registerFunctionCountEqual(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} diff --git a/dbms/src/Functions/currentDatabase.cpp b/dbms/src/Functions/currentDatabase.cpp new file mode 100644 index 00000000000..6c5137fe2ee --- /dev/null +++ b/dbms/src/Functions/currentDatabase.cpp @@ -0,0 +1,53 @@ +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionCurrentDatabase : public IFunction +{ + const String db_name; + +public: + static constexpr auto name = "currentDatabase"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getCurrentDatabase()); + } + + explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name); + } +}; + + +void registerFunctionCurrentDatabase(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/defaultValueOfArgumentType.cpp b/dbms/src/Functions/defaultValueOfArgumentType.cpp new file mode 100644 index 00000000000..452f25b51d8 --- /dev/null +++ b/dbms/src/Functions/defaultValueOfArgumentType.cpp @@ -0,0 +1,48 @@ +#include +#include + + +namespace DB +{ + +/// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String). +class FunctionDefaultValueOfArgumentType : public IFunction +{ +public: + static constexpr auto name = "defaultValueOfArgumentType"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const IDataType & type = *block.getByPosition(arguments[0]).type; + block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); + } +}; + + +void registerFunctionDefaultValueOfArgumentType(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/dumpColumnStructure.cpp b/dbms/src/Functions/dumpColumnStructure.cpp new file mode 100644 index 00000000000..95a2c37951b --- /dev/null +++ b/dbms/src/Functions/dumpColumnStructure.cpp @@ -0,0 +1,54 @@ +#include +#include +#include + + +namespace DB +{ + +/// Dump the structure of type and column. +class FunctionDumpColumnStructure : public IFunction +{ +public: + static constexpr auto name = "dumpColumnStructure"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & elem = block.getByPosition(arguments[0]); + + /// Note that the result is not a constant, because it contains block size. + + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, + elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst(); + } +}; + + +void registerFunctionDumpColumnStructure(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/empty.cpp b/dbms/src/Functions/empty.cpp new file mode 100644 index 00000000000..dfc13253b48 --- /dev/null +++ b/dbms/src/Functions/empty.cpp @@ -0,0 +1,22 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameEmpty +{ + static constexpr auto name = "empty"; +}; +using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8>; + +void registerFunctionEmpty(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/dbms/src/Functions/emptyArray.cpp b/dbms/src/Functions/emptyArray.cpp new file mode 100644 index 00000000000..1f904425012 --- /dev/null +++ b/dbms/src/Functions/emptyArray.cpp @@ -0,0 +1,92 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// TODO Make it simple. + +template struct TypeToColumnType { using ColumnType = ColumnVector; }; +template <> struct TypeToColumnType { using ColumnType = ColumnString; }; + +template struct DataTypeToName : TypeName { }; +template <> struct DataTypeToName { static std::string get() { return "Date"; } }; +template <> struct DataTypeToName { static std::string get() { return "DateTime"; } }; + + +template +struct FunctionEmptyArray : public IFunction +{ + static constexpr auto base_name = "emptyArray"; + static const String name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + +private: + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + using UnderlyingColumnType = typename TypeToColumnType::ColumnType; + + block.getByPosition(result).column = ColumnArray::create( + UnderlyingColumnType::create(), + ColumnArray::ColumnOffsets::create(input_rows_count, 0)); + } +}; + + +template +const String FunctionEmptyArray::name = FunctionEmptyArray::base_name + String(DataTypeToName::get()); + +using FunctionEmptyArrayUInt8 = FunctionEmptyArray; +using FunctionEmptyArrayUInt16 = FunctionEmptyArray; +using FunctionEmptyArrayUInt32 = FunctionEmptyArray; +using FunctionEmptyArrayUInt64 = FunctionEmptyArray; +using FunctionEmptyArrayInt8 = FunctionEmptyArray; +using FunctionEmptyArrayInt16 = FunctionEmptyArray; +using FunctionEmptyArrayInt32 = FunctionEmptyArray; +using FunctionEmptyArrayInt64 = FunctionEmptyArray; +using FunctionEmptyArrayFloat32 = FunctionEmptyArray; +using FunctionEmptyArrayFloat64 = FunctionEmptyArray; +using FunctionEmptyArrayDate = FunctionEmptyArray; +using FunctionEmptyArrayDateTime = FunctionEmptyArray; +using FunctionEmptyArrayString = FunctionEmptyArray; + + +void registerFunctionsEmptyArray(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/emptyArrayToSingle.cpp b/dbms/src/Functions/emptyArrayToSingle.cpp new file mode 100644 index 00000000000..90ed3df529b --- /dev/null +++ b/dbms/src/Functions/emptyArrayToSingle.cpp @@ -0,0 +1,425 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value. + */ +class FunctionEmptyArrayToSingle : public IFunction +{ +public: + static constexpr auto name = "emptyArrayToSingle"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception("Argument for function " + getName() + " must be array.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; +}; + + +namespace +{ + namespace FunctionEmptyArrayToSingleImpl + { + bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) + { + if (const ColumnConst * const_array = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) + { + if (const_array->getValue().empty()) + { + auto nested_type = typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType(); + + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( + input_rows_count, + Array{nested_type->getDefault()}); + } + else + block.getByPosition(result).column = block.getByPosition(arguments[0]).column; + + return true; + } + else + return false; + } + + template + bool executeNumber( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, ColumnArray::Offsets & res_offsets, + const NullMap * src_null_map, + NullMap * res_null_map) + { + if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) + { + const PaddedPODArray & src_data = src_data_concrete->getData(); + PaddedPODArray & res_data = static_cast &>(res_data_col).getData(); + + size_t size = src_offsets.size(); + res_offsets.resize(size); + res_data.reserve(src_data.size()); + + if (nullable) + res_null_map->reserve(src_null_map->size()); + + ColumnArray::Offset src_prev_offset = 0; + ColumnArray::Offset res_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_offsets[i] != src_prev_offset) + { + size_t size_to_write = src_offsets[i] - src_prev_offset; + res_data.resize(res_prev_offset + size_to_write); + memcpy(&res_data[res_prev_offset], &src_data[src_prev_offset], size_to_write * sizeof(T)); + + if (nullable) + { + res_null_map->resize(res_prev_offset + size_to_write); + memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); + } + + res_prev_offset += size_to_write; + res_offsets[i] = res_prev_offset; + } + else + { + res_data.push_back(T()); + ++res_prev_offset; + res_offsets[i] = res_prev_offset; + + if (nullable) + res_null_map->push_back(1); /// Push NULL. + } + + src_prev_offset = src_offsets[i]; + } + + return true; + } + else + return false; + } + + + template + bool executeFixedString( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data_col, ColumnArray::Offsets & res_offsets, + const NullMap * src_null_map, + NullMap * res_null_map) + { + if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) + { + const size_t n = src_data_concrete->getN(); + const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); + + auto concrete_res_data = typeid_cast(&res_data_col); + if (!concrete_res_data) + throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; + + ColumnFixedString::Chars_t & res_data = concrete_res_data->getChars(); + size_t size = src_offsets.size(); + res_offsets.resize(size); + res_data.reserve(src_data.size()); + + if (nullable) + res_null_map->reserve(src_null_map->size()); + + ColumnArray::Offset src_prev_offset = 0; + ColumnArray::Offset res_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_offsets[i] != src_prev_offset) + { + size_t size_to_write = src_offsets[i] - src_prev_offset; + size_t prev_res_data_size = res_data.size(); + res_data.resize(prev_res_data_size + size_to_write * n); + memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset * n], size_to_write * n); + + if (nullable) + { + res_null_map->resize(res_prev_offset + size_to_write); + memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); + } + + res_prev_offset += size_to_write; + res_offsets[i] = res_prev_offset; + } + else + { + size_t prev_res_data_size = res_data.size(); + res_data.resize(prev_res_data_size + n); + memset(&res_data[prev_res_data_size], 0, n); + ++res_prev_offset; + res_offsets[i] = res_prev_offset; + + if (nullable) + res_null_map->push_back(1); + } + + src_prev_offset = src_offsets[i]; + } + + return true; + } + else + return false; + } + + + template + bool executeString( + const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, + IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets, + const NullMap * src_null_map, + NullMap * res_null_map) + { + if (const ColumnString * src_data_concrete = checkAndGetColumn(&src_data)) + { + const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); + + auto concrete_res_string_offsets = typeid_cast(&res_data_col); + if (!concrete_res_string_offsets) + throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; + ColumnString::Offsets & res_string_offsets = concrete_res_string_offsets->getOffsets(); + + const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); + + auto concrete_res_data = typeid_cast(&res_data_col); + if (!concrete_res_data) + throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; + ColumnString::Chars_t & res_data = concrete_res_data->getChars(); + + size_t size = src_array_offsets.size(); + res_array_offsets.resize(size); + res_string_offsets.reserve(src_string_offsets.size()); + res_data.reserve(src_data.size()); + + if (nullable) + res_null_map->reserve(src_null_map->size()); + + ColumnArray::Offset src_array_prev_offset = 0; + ColumnArray::Offset res_array_prev_offset = 0; + + ColumnString::Offset src_string_prev_offset = 0; + ColumnString::Offset res_string_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_array_offsets[i] != src_array_prev_offset) + { + size_t array_size = src_array_offsets[i] - src_array_prev_offset; + + size_t bytes_to_copy = 0; + size_t from_string_prev_offset_local = src_string_prev_offset; + for (size_t j = 0; j < array_size; ++j) + { + size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local; + + res_string_prev_offset += string_size; + res_string_offsets.push_back(res_string_prev_offset); + + from_string_prev_offset_local += string_size; + bytes_to_copy += string_size; + } + + size_t res_data_old_size = res_data.size(); + res_data.resize(res_data_old_size + bytes_to_copy); + memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy); + + if (nullable) + { + res_null_map->resize(res_array_prev_offset + array_size); + memcpy(&(*res_null_map)[res_array_prev_offset], &(*src_null_map)[src_array_prev_offset], array_size); + } + + res_array_prev_offset += array_size; + res_array_offsets[i] = res_array_prev_offset; + } + else + { + res_data.push_back(0); /// An empty string, including zero at the end. + + if (nullable) + res_null_map->push_back(1); + + ++res_string_prev_offset; + res_string_offsets.push_back(res_string_prev_offset); + + ++res_array_prev_offset; + res_array_offsets[i] = res_array_prev_offset; + } + + src_array_prev_offset = src_array_offsets[i]; + + if (src_array_prev_offset) + src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1]; + } + + return true; + } + else + return false; + } + + + template + void executeGeneric( + const IColumn & src_data, const ColumnArray::Offsets & src_offsets, + IColumn & res_data, ColumnArray::Offsets & res_offsets, + const NullMap * src_null_map, + NullMap * res_null_map) + { + size_t size = src_offsets.size(); + res_offsets.resize(size); + res_data.reserve(src_data.size()); + + if (nullable) + res_null_map->reserve(src_null_map->size()); + + ColumnArray::Offset src_prev_offset = 0; + ColumnArray::Offset res_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_offsets[i] != src_prev_offset) + { + size_t size_to_write = src_offsets[i] - src_prev_offset; + res_data.insertRangeFrom(src_data, src_prev_offset, size_to_write); + + if (nullable) + { + res_null_map->resize(res_prev_offset + size_to_write); + memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write); + } + + res_prev_offset += size_to_write; + res_offsets[i] = res_prev_offset; + } + else + { + res_data.insertDefault(); + ++res_prev_offset; + res_offsets[i] = res_prev_offset; + + if (nullable) + res_null_map->push_back(1); + } + + src_prev_offset = src_offsets[i]; + } + } + + + template + void executeDispatch( + const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, + IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets, + const NullMap * src_null_map, + NullMap * res_null_map) + { + if (!( executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeNumber(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeString (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + || executeFixedString (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map))) + executeGeneric (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map); + } + } +} + + +void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count)) + return; + + const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); + if (!array) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + MutableColumnPtr res_ptr = array->cloneEmpty(); + ColumnArray & res = static_cast(*res_ptr); + + const IColumn & src_data = array->getData(); + const ColumnArray::Offsets & src_offsets = array->getOffsets(); + IColumn & res_data = res.getData(); + ColumnArray::Offsets & res_offsets = res.getOffsets(); + + const NullMap * src_null_map = nullptr; + NullMap * res_null_map = nullptr; + + const IColumn * inner_col; + IColumn * inner_res_col; + + bool nullable = src_data.isColumnNullable(); + if (nullable) + { + auto nullable_col = static_cast(&src_data); + inner_col = &nullable_col->getNestedColumn(); + src_null_map = &nullable_col->getNullMapData(); + + auto nullable_res_col = static_cast(&res_data); + inner_res_col = &nullable_res_col->getNestedColumn(); + res_null_map = &nullable_res_col->getNullMapData(); + } + else + { + inner_col = &src_data; + inner_res_col = &res_data; + } + + if (nullable) + FunctionEmptyArrayToSingleImpl::executeDispatch(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map); + else + FunctionEmptyArrayToSingleImpl::executeDispatch(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map); + + block.getByPosition(result).column = std::move(res_ptr); +} + + +void registerFunctionEmptyArrayToSingle(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/endsWith.cpp b/dbms/src/Functions/endsWith.cpp new file mode 100644 index 00000000000..40ea7902465 --- /dev/null +++ b/dbms/src/Functions/endsWith.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionEndsWith = FunctionStartsEndsWith; + +void registerFunctionEndsWith(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} + diff --git a/dbms/src/Functions/finalizeAggregation.cpp b/dbms/src/Functions/finalizeAggregation.cpp new file mode 100644 index 00000000000..3f7ba9eb4c5 --- /dev/null +++ b/dbms/src/Functions/finalizeAggregation.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** finalizeAggregation(agg_state) - get the result from the aggregation state. + * Takes state of aggregate function. Returns result of aggregation (finalized state). + */ +class FunctionFinalizeAggregation : public IFunction +{ +public: + static constexpr auto name = "finalizeAggregation"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); + if (!type) + throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnAggregateFunction * column_with_states + = typeid_cast(&*block.getByPosition(arguments.at(0)).column); + if (!column_with_states) + throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + + " of first argument of function " + + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + block.getByPosition(result).column = column_with_states->convertToValues(); + } +}; + + +void registerFunctionFinalizeAggregation(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/getSizeOfEnumType.cpp b/dbms/src/Functions/getSizeOfEnumType.cpp new file mode 100644 index 00000000000..4aef8492243 --- /dev/null +++ b/dbms/src/Functions/getSizeOfEnumType.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// Returns number of fields in Enum data type of passed value. +class FunctionGetSizeOfEnumType : public IFunction +{ +public: + static constexpr auto name = "getSizeOfEnumType"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + WhichDataType which(arguments[0]); + + if (which.isEnum8()) + return std::make_shared(); + else if (which.isEnum16()) + return std::make_shared(); + + throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(type->getValues().size())); + else if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) + block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, UInt64(type->getValues().size())); + else + throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + + +void registerFunctionGetSizeOfEnumType(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/has.cpp b/dbms/src/Functions/has.cpp new file mode 100644 index 00000000000..06820eef06a --- /dev/null +++ b/dbms/src/Functions/has.cpp @@ -0,0 +1,18 @@ +#include +#include + + +namespace DB +{ + +struct NameHas { static constexpr auto name = "has"; }; + +/// has(arr, x) - whether there is an element x in the array. +using FunctionHas = FunctionArrayIndex; + +void registerFunctionHas(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasAll.cpp b/dbms/src/Functions/hasAll.cpp new file mode 100644 index 00000000000..09784e7df10 --- /dev/null +++ b/dbms/src/Functions/hasAll.cpp @@ -0,0 +1,21 @@ +#include +#include + + +namespace DB +{ + +class FunctionArrayHasAll : public FunctionArrayHasAllAny +{ +public: + static constexpr auto name = "hasAll"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayHasAll(const Context & context) : FunctionArrayHasAllAny(context, true, name) {} +}; + +void registerFunctionHasAll(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasAllAny.h b/dbms/src/Functions/hasAllAny.h new file mode 100644 index 00000000000..45780d0aa3c --- /dev/null +++ b/dbms/src/Functions/hasAllAny.h @@ -0,0 +1,123 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionArrayHasAllAny : public IFunction +{ +public: + FunctionArrayHasAllAny(const Context & context, bool all, const char * name) + : context(context), all(all), name(name) {} + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + for (auto i : ext::range(0, arguments.size())) + { + auto array_type = typeid_cast(arguments[i].get()); + if (!array_type) + throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type " + + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t rows = input_rows_count; + size_t num_args = arguments.size(); + + auto result_column = ColumnUInt8::create(rows); + + DataTypePtr common_type = nullptr; + auto commonType = [& common_type, & block, & arguments]() + { + if (common_type == nullptr) + { + DataTypes data_types; + data_types.reserve(arguments.size()); + for (const auto & argument : arguments) + data_types.push_back(block.getByPosition(argument).type); + + common_type = getLeastSupertype(data_types); + } + + return common_type; + }; + + Columns preprocessed_columns(num_args); + + for (size_t i = 0; i < num_args; ++i) + { + const auto & argument = block.getByPosition(arguments[i]); + ColumnPtr preprocessed_column = argument.column; + + const auto argument_type = typeid_cast(argument.type.get()); + const auto & nested_type = argument_type->getNestedType(); + + /// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1 + if (typeid_cast(removeNullable(nested_type).get())) + preprocessed_column = castColumn(argument, commonType(), context); + + preprocessed_columns[i] = std::move(preprocessed_column); + } + + std::vector> sources; + + for (auto & argument_column : preprocessed_columns) + { + bool is_const = false; + + if (auto argument_column_const = typeid_cast(argument_column.get())) + { + is_const = true; + argument_column = argument_column_const->getDataColumnPtr(); + } + + if (auto argument_column_array = typeid_cast(argument_column.get())) + sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows)); + else + throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR}; + } + + auto result_column_ptr = typeid_cast(result_column.get()); + GatherUtils::sliceHas(*sources[0], *sources[1], all, *result_column_ptr); + + block.getByPosition(result).column = std::move(result_column); + } + + bool useDefaultImplementationForConstants() const override { return true; } + +private: + const Context & context; + bool all; + const char * name; +}; + +} diff --git a/dbms/src/Functions/hasAny.cpp b/dbms/src/Functions/hasAny.cpp new file mode 100644 index 00000000000..7ff8731b014 --- /dev/null +++ b/dbms/src/Functions/hasAny.cpp @@ -0,0 +1,21 @@ +#include +#include + + +namespace DB +{ + +class FunctionArrayHasAny : public FunctionArrayHasAllAny +{ +public: + static constexpr auto name = "hasAny"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayHasAny(const Context & context) : FunctionArrayHasAllAny(context, false, name) {} +}; + +void registerFunctionHasAny(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasColumnInTable.cpp b/dbms/src/Functions/hasColumnInTable.cpp new file mode 100644 index 00000000000..5d539281bf6 --- /dev/null +++ b/dbms/src/Functions/hasColumnInTable.cpp @@ -0,0 +1,144 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** Usage: + * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') + */ +class FunctionHasColumnInTable : public IFunction +{ +public: + static constexpr auto name = "hasColumnInTable"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getGlobalContext()); + } + + explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) + { + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + String getName() const override + { + return name; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + const Context & global_context; +}; + + +DataTypePtr FunctionHasColumnInTable::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + if (arguments.size() < 3 || arguments.size() > 6) + throw Exception{"Invalid number of arguments for function " + getName(), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + static const std::string arg_pos_description[] = {"First", "Second", "Third", "Fourth", "Fifth", "Sixth"}; + for (size_t i = 0; i < arguments.size(); ++i) + { + const ColumnWithTypeAndName & argument = arguments[i]; + + if (!checkColumnConst(argument.column.get())) + { + throw Exception(arg_pos_description[i] + " argument for function " + getName() + " must be const String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + return std::make_shared(); +} + + +void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + auto get_string_from_block = [&](size_t column_pos) -> String + { + ColumnPtr column = block.getByPosition(column_pos).column; + const ColumnConst * const_column = checkAndGetColumnConst(column.get()); + return const_column->getValue(); + }; + + size_t arg = 0; + String host_name; + String user_name; + String password; + + if (arguments.size() > 3) + host_name = get_string_from_block(arguments[arg++]); + + if (arguments.size() > 4) + user_name = get_string_from_block(arguments[arg++]); + + if (arguments.size() > 5) + password = get_string_from_block(arguments[arg++]); + + String database_name = get_string_from_block(arguments[arg++]); + String table_name = get_string_from_block(arguments[arg++]); + String column_name = get_string_from_block(arguments[arg++]); + + bool has_column; + if (host_name.empty()) + { + const StoragePtr & table = global_context.getTable(database_name, table_name); + has_column = table->hasColumn(column_name); + } + else + { + std::vector> host_names = {{ host_name }}; + + auto cluster = std::make_shared( + global_context.getSettings(), + host_names, + !user_name.empty() ? user_name : "default", + password, + global_context.getTCPPort(), + false); + + auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context); + has_column = remote_columns.hasPhysical(column_name); + } + + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(has_column)); +} + + +void registerFunctionHasColumnInTable(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hostName.cpp b/dbms/src/Functions/hostName.cpp new file mode 100644 index 00000000000..489305c3108 --- /dev/null +++ b/dbms/src/Functions/hostName.cpp @@ -0,0 +1,58 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/// Get the host name. Is is constant on single server, but is not constant in distributed queries. +class FunctionHostName : public IFunction +{ +public: + static constexpr auto name = "hostName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + /** convertToFullColumn needed because in distributed query processing, + * each server returns its own value. + */ + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( + input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst(); + } +}; + + +void registerFunctionHostName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/identity.cpp b/dbms/src/Functions/identity.cpp new file mode 100644 index 00000000000..24977091d07 --- /dev/null +++ b/dbms/src/Functions/identity.cpp @@ -0,0 +1,44 @@ +#include +#include + + +namespace DB +{ + +class FunctionIdentity : public IFunction +{ +public: + static constexpr auto name = "identity"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments.front(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + block.getByPosition(result).column = block.getByPosition(arguments.front()).column; + } +}; + + +void registerFunctionIdentity(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/ifNull.cpp b/dbms/src/Functions/ifNull.cpp new file mode 100644 index 00000000000..bf517293409 --- /dev/null +++ b/dbms/src/Functions/ifNull.cpp @@ -0,0 +1,100 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function ifNull which takes 2 arguments and returns +/// the value of the 1st argument if it is not null. Otherwise it returns +/// the value of the 2nd argument. +class FunctionIfNull : public IFunction +{ +public: + static constexpr auto name = "ifNull"; + + FunctionIfNull(const Context & context) : context(context) {} + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments[0]->onlyNull()) + return arguments[1]; + + if (!arguments[0]->isNullable()) + return arguments[0]; + + return getLeastSupertype({removeNullable(arguments[0]), arguments[1]}); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + /// Always null. + if (block.getByPosition(arguments[0]).type->onlyNull()) + { + block.getByPosition(result).column = block.getByPosition(arguments[1]).column; + return; + } + + /// Could not contain nulls, so nullIf makes no sense. + if (!block.getByPosition(arguments[0]).type->isNullable()) + { + block.getByPosition(result).column = block.getByPosition(arguments[0]).column; + return; + } + + /// ifNull(col1, col2) == if(isNotNull(col1), assumeNotNull(col1), col2) + + Block temp_block = block; + + size_t is_not_null_pos = temp_block.columns(); + temp_block.insert({nullptr, std::make_shared(), ""}); + size_t assume_not_null_pos = temp_block.columns(); + temp_block.insert({nullptr, removeNullable(block.getByPosition(arguments[0]).type), ""}); + + auto is_not_null = FunctionFactory::instance().get("isNotNull", context)->build( + {temp_block.getByPosition(arguments[0])}); + + auto assume_not_null = FunctionFactory::instance().get("assumeNotNull", context)->build( + {temp_block.getByPosition(arguments[0])}); + + auto func_if = FunctionFactory::instance().get("if", context)->build( + {temp_block.getByPosition(is_not_null_pos), temp_block.getByPosition(assume_not_null_pos), temp_block.getByPosition(arguments[1])}); + + is_not_null->execute(temp_block, {arguments[0]}, is_not_null_pos, input_rows_count); + assume_not_null->execute(temp_block, {arguments[0]}, assume_not_null_pos, input_rows_count); + func_if->execute(temp_block, {is_not_null_pos, assume_not_null_pos, arguments[1]}, result, input_rows_count); + + block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + } + +private: + const Context & context; +}; + + +void registerFunctionIfNull(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/ignore.cpp b/dbms/src/Functions/ignore.cpp new file mode 100644 index 00000000000..09a7ac2653f --- /dev/null +++ b/dbms/src/Functions/ignore.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + +/** ignore(...) is a function that takes any arguments, and always returns 0. + */ +class FunctionIgnore : public IFunction +{ +public: + static constexpr auto name = "ignore"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + String getName() const override + { + return name; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); + } +}; + + +void registerFunctionIgnore(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp new file mode 100644 index 00000000000..9a984da1bac --- /dev/null +++ b/dbms/src/Functions/in.cpp @@ -0,0 +1,131 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** in(x, set) - function for evaluating the IN + * notIn(x, set) - and NOT IN. + */ + +template +struct FunctionInName; + +template <> +struct FunctionInName +{ + static constexpr auto name = "in"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "globalIn"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "notIn"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "globalNotIn"; +}; + +template +class FunctionIn : public IFunction +{ +public: + static constexpr auto name = FunctionInName::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool useDefaultImplementationForNulls() const override + { + return false; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + /// Second argument must be ColumnSet. + ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; + const ColumnSet * column_set = typeid_cast(&*column_set_ptr); + if (!column_set) + throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), + ErrorCodes::ILLEGAL_COLUMN); + + Block block_of_key_columns; + + /// First argument may be tuple or single column. + const ColumnWithTypeAndName & left_arg = block.getByPosition(arguments[0]); + const ColumnTuple * tuple = typeid_cast(left_arg.column.get()); + const ColumnConst * const_tuple = checkAndGetColumnConst(left_arg.column.get()); + const DataTypeTuple * type_tuple = typeid_cast(left_arg.type.get()); + + ColumnPtr materialized_tuple; + if (const_tuple) + { + materialized_tuple = const_tuple->convertToFullColumn(); + tuple = typeid_cast(materialized_tuple.get()); + } + + auto set = column_set->getData(); + auto set_types = set->getDataTypes(); + if (tuple && (set_types.size() != 1 || !set_types[0]->equals(*type_tuple))) + { + const Columns & tuple_columns = tuple->getColumns(); + const DataTypes & tuple_types = type_tuple->getElements(); + size_t tuple_size = tuple_columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + block_of_key_columns.insert({ tuple_columns[i], tuple_types[i], "" }); + } + else + block_of_key_columns.insert(left_arg); + + block.getByPosition(result).column = set->execute(block_of_key_columns, negative); + } +}; + + +void registerFunctionsIn(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/indexHint.cpp b/dbms/src/Functions/indexHint.cpp new file mode 100644 index 00000000000..2b1e2608df6 --- /dev/null +++ b/dbms/src/Functions/indexHint.cpp @@ -0,0 +1,63 @@ +#include +#include +#include + + +namespace DB +{ + + +/** The `indexHint` function takes any number of any arguments and always returns one. + * + * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) + * - the expressions inside it are not evaluated; + * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, + * as if instead of using it the expression itself would be. + * + * Example: WHERE something AND indexHint(CounterID = 34) + * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. + * + * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. + */ +class FunctionIndexHint : public IFunction +{ +public: + static constexpr auto name = "indexHint"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + String getName() const override + { + return name; + } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(1)); + } +}; + + +void registerFunctionIndexHint(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/indexOf.cpp b/dbms/src/Functions/indexOf.cpp new file mode 100644 index 00000000000..a095e550349 --- /dev/null +++ b/dbms/src/Functions/indexOf.cpp @@ -0,0 +1,19 @@ +#include +#include + + +namespace DB +{ + +struct NameIndexOf { static constexpr auto name = "indexOf"; }; + +/// indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not. +using FunctionIndexOf = FunctionArrayIndex; + +void registerFunctionIndexOf(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} diff --git a/dbms/src/Functions/isFinite.cpp b/dbms/src/Functions/isFinite.cpp new file mode 100644 index 00000000000..e898ddc8304 --- /dev/null +++ b/dbms/src/Functions/isFinite.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct IsFiniteImpl +{ + /// Better implementation, because isinf, isfinite, isnan are not inlined for unknown reason. + /// Assuming IEEE 754. + /// NOTE gcc 7 doesn't vectorize this loop. + + static constexpr auto name = "isFinite"; + template + static bool execute(const T t) + { + if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b01111111100000000000000000000000) + != 0b01111111100000000000000000000000; + else if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b0111111111110000000000000000000000000000000000000000000000000000) + != 0b0111111111110000000000000000000000000000000000000000000000000000; + else + { + (void)t; + return true; + } + } +}; + +using FunctionIsFinite = FunctionNumericPredicate; + + +void registerFunctionIsFinite(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isInfinite.cpp b/dbms/src/Functions/isInfinite.cpp new file mode 100644 index 00000000000..17208373cb3 --- /dev/null +++ b/dbms/src/Functions/isInfinite.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct IsInfiniteImpl +{ + static constexpr auto name = "isInfinite"; + template + static bool execute(const T t) + { + if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b01111111111111111111111111111111) + == 0b01111111100000000000000000000000; + else if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b0111111111111111111111111111111111111111111111111111111111111111) + == 0b0111111111110000000000000000000000000000000000000000000000000000; + else + { + (void)t; + return false; + } + } +}; + +using FunctionIsInfinite = FunctionNumericPredicate; + + +void registerFunctionIsInfinite(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isNaN.cpp b/dbms/src/Functions/isNaN.cpp new file mode 100644 index 00000000000..5146f7cfc0d --- /dev/null +++ b/dbms/src/Functions/isNaN.cpp @@ -0,0 +1,27 @@ +#include +#include + + +namespace DB +{ + +struct IsNaNImpl +{ + static constexpr auto name = "isNaN"; + template + static bool execute(const T t) + { + /// Suppression for PVS-Studio. + return t != t; //-V501 + } +}; + +using FunctionIsNaN = FunctionNumericPredicate; + + +void registerFunctionIsNaN(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isNotNull.cpp b/dbms/src/Functions/isNotNull.cpp new file mode 100644 index 00000000000..fe45d054cb9 --- /dev/null +++ b/dbms/src/Functions/isNotNull.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function isNotNull which returns true if a value +/// is not null, false otherwise. +class FunctionIsNotNull : public IFunction +{ +public: + static constexpr auto name = "isNotNull"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes &) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnWithTypeAndName & elem = block.getByPosition(arguments[0]); + if (elem.column->isColumnNullable()) + { + /// Return the negated null map. + auto res_column = ColumnUInt8::create(input_rows_count); + const auto & src_data = static_cast(*elem.column).getNullMapData(); + auto & res_data = static_cast(*res_column).getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = !src_data[i]; + + block.getByPosition(result).column = std::move(res_column); + } + else + { + /// Since no element is nullable, return a constant one. + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(elem.column->size(), UInt64(1)); + } + } +}; + +void registerFunctionIsNotNull(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isNull.cpp b/dbms/src/Functions/isNull.cpp new file mode 100644 index 00000000000..2d77f4d5c79 --- /dev/null +++ b/dbms/src/Functions/isNull.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function isNull which returns true if a value +/// is null, false otherwise. +class FunctionIsNull : public IFunction +{ +public: + static constexpr auto name = "isNull"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes &) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + const ColumnWithTypeAndName & elem = block.getByPosition(arguments[0]); + if (elem.column->isColumnNullable()) + { + /// Merely return the embedded null map. + block.getByPosition(result).column = static_cast(*elem.column).getNullMapColumnPtr(); + } + else + { + /// Since no element is nullable, return a zero-constant column representing + /// a zero-filled null map. + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(elem.column->size(), UInt64(0)); + } + } +}; + + +void registerFunctionIsNull(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/length.cpp b/dbms/src/Functions/length.cpp new file mode 100644 index 00000000000..653e3166ba9 --- /dev/null +++ b/dbms/src/Functions/length.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + + +/** Calculates the length of a string in bytes. + */ +struct LengthImpl +{ + static constexpr auto is_fixed_to_constant = true; + + static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + for (size_t i = 0; i < size; ++i) + res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); + } + + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res) + { + res = n; + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) + { + } + + static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + for (size_t i = 0; i < size; ++i) + res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]); + } +}; + + +struct NameLength +{ + static constexpr auto name = "length"; +}; + +using FunctionLength = FunctionStringOrArrayToT; + +void registerFunctionLength(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lengthUTF8.cpp b/dbms/src/Functions/lengthUTF8.cpp new file mode 100644 index 00000000000..734f62afb40 --- /dev/null +++ b/dbms/src/Functions/lengthUTF8.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** If the string is UTF-8 encoded text, it returns the length of the text in code points. + * (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization) + * (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization) + * Otherwise, the behavior is undefined. + */ +struct LengthUTF8Impl +{ + static constexpr auto is_fixed_to_constant = false; + + static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = UTF8::countCodePoints(&data[prev_offset], offsets[i] - prev_offset - 1); + prev_offset = offsets[i]; + } + } + + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/) + { + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + { + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + { + res[i] = UTF8::countCodePoints(&data[i * n], n); + } + } + + static void array(const ColumnString::Offsets &, PaddedPODArray &) + { + throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + +struct NameLengthUTF8 +{ + static constexpr auto name = "lengthUTF8"; +}; +using FunctionLengthUTF8 = FunctionStringOrArrayToT; + +void registerFunctionLengthUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowCardinalityIndices.cpp b/dbms/src/Functions/lowCardinalityIndices.cpp new file mode 100644 index 00000000000..1a0c05d738c --- /dev/null +++ b/dbms/src/Functions/lowCardinalityIndices.cpp @@ -0,0 +1,63 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionLowCardinalityIndices: public IFunction +{ +public: + static constexpr auto name = "lowCardinalityIndices"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); + auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); + auto & data = new_indexes_col->getData(); + for (size_t i = 0; i < data.size(); ++i) + data[i] = indexes_col->getUInt(i); + + res.column = std::move(new_indexes_col); + } +}; + + +void registerFunctionLowCardinalityIndices(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowCardinalityKeys.cpp b/dbms/src/Functions/lowCardinalityKeys.cpp new file mode 100644 index 00000000000..e4654a35a68 --- /dev/null +++ b/dbms/src/Functions/lowCardinalityKeys.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionLowCardinalityKeys: public IFunction +{ +public: + static constexpr auto name = "lowCardinalityKeys"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getDictionaryType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + const auto * column_with_dictionary = typeid_cast(arg.column.get()); + res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); + } +}; + + +void registerFunctionLowCardinalityKeys(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lower.cpp b/dbms/src/Functions/lower.cpp new file mode 100644 index 00000000000..38379fa10cd --- /dev/null +++ b/dbms/src/Functions/lower.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameLower +{ + static constexpr auto name = "lower"; +}; +using FunctionLower = FunctionStringToString, NameLower>; + +void registerFunctionLower(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowerUTF8.cpp b/dbms/src/Functions/lowerUTF8.cpp new file mode 100644 index 00000000000..fed031b524c --- /dev/null +++ b/dbms/src/Functions/lowerUTF8.cpp @@ -0,0 +1,23 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct NameLowerUTF8 +{ + static constexpr auto name = "lowerUTF8"; +}; + +using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; + +void registerFunctionLowerUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/materialize.cpp b/dbms/src/Functions/materialize.cpp new file mode 100644 index 00000000000..4d3a1a57a77 --- /dev/null +++ b/dbms/src/Functions/materialize.cpp @@ -0,0 +1,51 @@ +#include +#include + + +namespace DB +{ + +/** materialize(x) - materialize the constant + */ +class FunctionMaterialize : public IFunction +{ +public: + static constexpr auto name = "materialize"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto & src = block.getByPosition(arguments[0]).column; + if (ColumnPtr converted = src->convertToFullColumnIfConst()) + block.getByPosition(result).column = converted; + else + block.getByPosition(result).column = src; + } +}; + + +void registerFunctionMaterialize(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/notEmpty.cpp b/dbms/src/Functions/notEmpty.cpp new file mode 100644 index 00000000000..d8057501848 --- /dev/null +++ b/dbms/src/Functions/notEmpty.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameNotEmpty +{ + static constexpr auto name = "notEmpty"; +}; +using FunctionNotEmpty = FunctionStringOrArrayToT, NameNotEmpty, UInt8>; + +void registerFunctionNotEmpty(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp new file mode 100644 index 00000000000..0039dbad2b1 --- /dev/null +++ b/dbms/src/Functions/nullIf.cpp @@ -0,0 +1,86 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Implements the function nullIf which takes 2 arguments and returns +/// NULL if both arguments have the same value. Otherwise it returns the +/// value of the first argument. +class FunctionNullIf : public IFunction +{ +private: + const Context & context; +public: + static constexpr auto name = "nullIf"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + FunctionNullIf(const Context & context) : context(context) {} + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return makeNullable(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + /// nullIf(col1, col2) == if(col1 == col2, NULL, col1) + + Block temp_block = block; + + size_t res_pos = temp_block.columns(); + temp_block.insert({nullptr, std::make_shared(), ""}); + + { + auto equals_func = FunctionFactory::instance().get("equals", context)->build( + {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); + equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count); + } + + /// Argument corresponding to the NULL value. + size_t null_pos = temp_block.columns(); + + /// Append a NULL column. + ColumnWithTypeAndName null_elem; + null_elem.type = block.getByPosition(result).type; + null_elem.column = null_elem.type->createColumnConstWithDefaultValue(input_rows_count); + null_elem.name = "NULL"; + + temp_block.insert(null_elem); + + auto func_if = FunctionFactory::instance().get("if", context)->build( + {temp_block.getByPosition(res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); + func_if->execute(temp_block, {res_pos, null_pos, arguments[0]}, result, input_rows_count); + + block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + } +}; + + + +void registerFunctionNullIf(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} + diff --git a/dbms/src/Functions/range.cpp b/dbms/src/Functions/range.cpp new file mode 100644 index 00000000000..9a6f502beb8 --- /dev/null +++ b/dbms/src/Functions/range.cpp @@ -0,0 +1,111 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionRange : public IFunction +{ +public: + static constexpr auto name = "range"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + +private: + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypePtr & arg = arguments.front(); + + if (!isUnsignedInteger(arg)) + throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(arg); + } + + template + bool executeInternal(Block & block, const IColumn * arg, const size_t result) + { + static constexpr size_t max_elements = 100'000'000; + + if (const auto in = checkAndGetColumn>(arg)) + { + const auto & in_data = in->getData(); + const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{}, + [this] (const size_t lhs, const size_t rhs) + { + const auto sum = lhs + rhs; + if (sum < lhs) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + return sum; + }); + + if (total_values > max_elements) + throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(in->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + IColumn::Offset offset{}; + for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx) + { + for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx) + out_data[offset + elem_idx] = elem_idx; + + offset += in_data[row_idx]; + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + else + return false; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + const auto col = block.getByPosition(arguments[0]).column.get(); + + if (!executeInternal(block, col, result) && + !executeInternal(block, col, result) && + !executeInternal(block, col, result) && + !executeInternal(block, col, result)) + { + throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + } +}; + + +void registerFunctionRange(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index a230984b6ee..c7f1ca82cab 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -1,5 +1,5 @@ +#include #include - #include @@ -38,11 +38,14 @@ void registerFunctionsVisitParam(FunctionFactory &); void registerFunctionsMath(FunctionFactory &); void registerFunctionsTransform(FunctionFactory &); void registerFunctionsGeo(FunctionFactory &); -void registerFunctionsCharset(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); void registerFunctionsProjection(FunctionFactory &); +#if USE_ICU +void registerFunctionConvertCharset(FunctionFactory &); +#endif + void registerFunctions() { auto & factory = FunctionFactory::instance(); @@ -75,10 +78,13 @@ void registerFunctions() registerFunctionsMath(factory); registerFunctionsTransform(factory); registerFunctionsGeo(factory); - registerFunctionsCharset(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); registerFunctionsProjection(factory); + +#if USE_ICU + registerFunctionConvertCharset(factory); +#endif } } diff --git a/dbms/src/Functions/registerFunctionsArray.cpp b/dbms/src/Functions/registerFunctionsArray.cpp new file mode 100644 index 00000000000..0cf80bc4f73 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsArray.cpp @@ -0,0 +1,62 @@ +#include + +namespace DB +{ + +void registerFunctionArray(FunctionFactory &); +void registerFunctionArrayElement(FunctionFactory &); +void registerFunctionArrayResize(FunctionFactory &); +void registerFunctionHas(FunctionFactory &); +void registerFunctionHasAll(FunctionFactory &); +void registerFunctionHasAny(FunctionFactory &); +void registerFunctionIndexOf(FunctionFactory &); +void registerFunctionCountEqual(FunctionFactory &); +void registerFunctionArrayIntersect(FunctionFactory &); +void registerFunctionArrayPushFront(FunctionFactory &); +void registerFunctionArrayPushBack(FunctionFactory &); +void registerFunctionArrayPopFront(FunctionFactory &); +void registerFunctionArrayPopBack(FunctionFactory &); +void registerFunctionArrayConcat(FunctionFactory &); +void registerFunctionArraySlice(FunctionFactory &); +void registerFunctionArrayReverse(FunctionFactory &); +void registerFunctionArrayReduce(FunctionFactory &); +void registerFunctionRange(FunctionFactory &); +void registerFunctionsEmptyArray(FunctionFactory &); +void registerFunctionEmptyArrayToSingle(FunctionFactory &); +void registerFunctionArrayEnumerate(FunctionFactory &); +void registerFunctionArrayEnumerateUniq(FunctionFactory &); +void registerFunctionArrayEnumerateDense(FunctionFactory &); +void registerFunctionArrayUniq(FunctionFactory &); +void registerFunctionArrayDistinct(FunctionFactory &); + +void registerFunctionsArray(FunctionFactory & factory) +{ + registerFunctionArray(factory); + registerFunctionArrayElement(factory); + registerFunctionArrayResize(factory); + registerFunctionHas(factory); + registerFunctionHasAll(factory); + registerFunctionHasAny(factory); + registerFunctionIndexOf(factory); + registerFunctionCountEqual(factory); + registerFunctionArrayIntersect(factory); + registerFunctionArrayPushFront(factory); + registerFunctionArrayPushBack(factory); + registerFunctionArrayPopFront(factory); + registerFunctionArrayPopBack(factory); + registerFunctionArrayConcat(factory); + registerFunctionArraySlice(factory); + registerFunctionArrayReverse(factory); + registerFunctionArrayReduce(factory); + registerFunctionRange(factory); + registerFunctionsEmptyArray(factory); + registerFunctionEmptyArrayToSingle(factory); + registerFunctionArrayEnumerate(factory); + registerFunctionArrayEnumerateUniq(factory); + registerFunctionArrayEnumerateDense(factory); + registerFunctionArrayUniq(factory); + registerFunctionArrayDistinct(factory); +} + +} + diff --git a/dbms/src/Functions/FunctionsComparison.cpp b/dbms/src/Functions/registerFunctionsComparison.cpp similarity index 100% rename from dbms/src/Functions/FunctionsComparison.cpp rename to dbms/src/Functions/registerFunctionsComparison.cpp diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp new file mode 100644 index 00000000000..e6fe5ca1524 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -0,0 +1,85 @@ +#include + +namespace DB +{ + +void registerFunctionCurrentDatabase(FunctionFactory &); +void registerFunctionHostName(FunctionFactory &); +void registerFunctionVisibleWidth(FunctionFactory &); +void registerFunctionToTypeName(FunctionFactory &); +void registerFunctionGetSizeOfEnumType(FunctionFactory &); +void registerFunctionToColumnTypeName(FunctionFactory &); +void registerFunctionDumpColumnStructure(FunctionFactory &); +void registerFunctionDefaultValueOfArgumentType(FunctionFactory &); +void registerFunctionBlockSize(FunctionFactory &); +void registerFunctionBlockNumber(FunctionFactory &); +void registerFunctionRowNumberInBlock(FunctionFactory &); +void registerFunctionRowNumberInAllBlocks(FunctionFactory &); +void registerFunctionSleep(FunctionFactory &); +void registerFunctionSleepEachRow(FunctionFactory &); +void registerFunctionMaterialize(FunctionFactory &); +void registerFunctionIgnore(FunctionFactory &); +void registerFunctionIndexHint(FunctionFactory &); +void registerFunctionIdentity(FunctionFactory &); +void registerFunctionArrayJoin(FunctionFactory &); +void registerFunctionReplicate(FunctionFactory &); +void registerFunctionBar(FunctionFactory &); +void registerFunctionHasColumnInTable(FunctionFactory &); +void registerFunctionIsFinite(FunctionFactory &); +void registerFunctionIsInfinite(FunctionFactory &); +void registerFunctionIsNaN(FunctionFactory &); +void registerFunctionThrowIf(FunctionFactory &); +void registerFunctionVersion(FunctionFactory &); +void registerFunctionUptime(FunctionFactory &); +void registerFunctionTimeZone(FunctionFactory &); +void registerFunctionRunningAccumulate(FunctionFactory &); +void registerFunctionRunningDifference(FunctionFactory &); +void registerFunctionRunningDifferenceStartingWithFirstValue(FunctionFactory &); +void registerFunctionFinalizeAggregation(FunctionFactory &); +void registerFunctionToLowCardinality(FunctionFactory &); +void registerFunctionLowCardinalityIndices(FunctionFactory &); +void registerFunctionLowCardinalityKeys(FunctionFactory &); +void registerFunctionsIn(FunctionFactory &); + +void registerFunctionsMiscellaneous(FunctionFactory & factory) +{ + registerFunctionCurrentDatabase(factory); + registerFunctionHostName(factory); + registerFunctionVisibleWidth(factory); + registerFunctionToTypeName(factory); + registerFunctionGetSizeOfEnumType(factory); + registerFunctionToColumnTypeName(factory); + registerFunctionDumpColumnStructure(factory); + registerFunctionDefaultValueOfArgumentType(factory); + registerFunctionBlockSize(factory); + registerFunctionBlockNumber(factory); + registerFunctionRowNumberInBlock(factory); + registerFunctionRowNumberInAllBlocks(factory); + registerFunctionSleep(factory); + registerFunctionSleepEachRow(factory); + registerFunctionMaterialize(factory); + registerFunctionIgnore(factory); + registerFunctionIndexHint(factory); + registerFunctionIdentity(factory); + registerFunctionArrayJoin(factory); + registerFunctionReplicate(factory); + registerFunctionBar(factory); + registerFunctionHasColumnInTable(factory); + registerFunctionIsFinite(factory); + registerFunctionIsInfinite(factory); + registerFunctionIsNaN(factory); + registerFunctionThrowIf(factory); + registerFunctionVersion(factory); + registerFunctionUptime(factory); + registerFunctionTimeZone(factory); + registerFunctionRunningAccumulate(factory); + registerFunctionRunningDifference(factory); + registerFunctionRunningDifferenceStartingWithFirstValue(factory); + registerFunctionFinalizeAggregation(factory); + registerFunctionToLowCardinality(factory); + registerFunctionLowCardinalityIndices(factory); + registerFunctionLowCardinalityKeys(factory); + registerFunctionsIn(factory); +} + +} diff --git a/dbms/src/Functions/registerFunctionsNull.cpp b/dbms/src/Functions/registerFunctionsNull.cpp new file mode 100644 index 00000000000..e97c128018e --- /dev/null +++ b/dbms/src/Functions/registerFunctionsNull.cpp @@ -0,0 +1,26 @@ +#include + +namespace DB +{ + +void registerFunctionIsNull(FunctionFactory & factory); +void registerFunctionIsNotNull(FunctionFactory & factory); +void registerFunctionCoalesce(FunctionFactory & factory); +void registerFunctionIfNull(FunctionFactory & factory); +void registerFunctionNullIf(FunctionFactory & factory); +void registerFunctionAssumeNotNull(FunctionFactory & factory); +void registerFunctionToNullable(FunctionFactory & factory); + +void registerFunctionsNull(FunctionFactory & factory) +{ + registerFunctionIsNull(factory); + registerFunctionIsNotNull(factory); + registerFunctionCoalesce(factory); + registerFunctionIfNull(factory); + registerFunctionNullIf(factory); + registerFunctionAssumeNotNull(factory); + registerFunctionToNullable(factory); +} + +} + diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp new file mode 100644 index 00000000000..9adf3a52824 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -0,0 +1,44 @@ +#include + +namespace DB +{ + +void registerFunctionEmpty(FunctionFactory &); +void registerFunctionNotEmpty(FunctionFactory &); +void registerFunctionLength(FunctionFactory &); +void registerFunctionLengthUTF8(FunctionFactory &); +void registerFunctionLower(FunctionFactory &); +void registerFunctionUpper(FunctionFactory &); +void registerFunctionLowerUTF8(FunctionFactory &); +void registerFunctionUpperUTF8(FunctionFactory &); +void registerFunctionReverse(FunctionFactory &); +void registerFunctionReverseUTF8(FunctionFactory &); +void registerFunctionsConcat(FunctionFactory &); +void registerFunctionSubstring(FunctionFactory &); +void registerFunctionSubstringUTF8(FunctionFactory &); +void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); +void registerFunctionStartsWith(FunctionFactory &); +void registerFunctionEndsWith(FunctionFactory &); + +void registerFunctionsString(FunctionFactory & factory) +{ + registerFunctionEmpty(factory); + registerFunctionNotEmpty(factory); + registerFunctionLength(factory); + registerFunctionLengthUTF8(factory); + registerFunctionLower(factory); + registerFunctionUpper(factory); + registerFunctionLowerUTF8(factory); + registerFunctionUpperUTF8(factory); + registerFunctionReverse(factory); + registerFunctionReverseUTF8(factory); + registerFunctionsConcat(factory); + registerFunctionSubstring(factory); + registerFunctionSubstringUTF8(factory); + registerFunctionAppendTrailingCharIfAbsent(factory); + registerFunctionStartsWith(factory); + registerFunctionEndsWith(factory); +} + +} + diff --git a/dbms/src/Functions/registerFunctionsTuple.cpp b/dbms/src/Functions/registerFunctionsTuple.cpp new file mode 100644 index 00000000000..051ac9a6384 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsTuple.cpp @@ -0,0 +1,15 @@ +#include + +namespace DB +{ + +void registerFunctionTuple(FunctionFactory &); +void registerFunctionTupleElement(FunctionFactory &); + +void registerFunctionsTuple(FunctionFactory & factory) +{ + registerFunctionTuple(factory); + registerFunctionTupleElement(factory); +} + +} diff --git a/dbms/src/Functions/replicate.cpp b/dbms/src/Functions/replicate.cpp new file mode 100644 index 00000000000..756b745ccde --- /dev/null +++ b/dbms/src/Functions/replicate.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Creates an array, multiplying the column (the first argument) by the number of elements in the array (the second argument). + */ +class FunctionReplicate : public IFunction +{ +public: + static constexpr auto name = "replicate"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); + if (!array_type) + throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return std::make_shared(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + ColumnPtr first_column = block.getByPosition(arguments[0]).column; + const ColumnArray * array_column = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); + ColumnPtr temp_column; + if (!array_column) + { + auto const_array_column = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); + if (!const_array_column) + throw Exception("Unexpected column for replicate", ErrorCodes::ILLEGAL_COLUMN); + temp_column = const_array_column->convertToFullColumn(); + array_column = checkAndGetColumn(temp_column.get()); + } + block.getByPosition(result).column + = ColumnArray::create(first_column->replicate(array_column->getOffsets()), array_column->getOffsetsPtr()); + } +}; + + +void registerFunctionReplicate(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/reverse.cpp b/dbms/src/Functions/reverse.cpp new file mode 100644 index 00000000000..e9327f58f61 --- /dev/null +++ b/dbms/src/Functions/reverse.cpp @@ -0,0 +1,153 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** Reverse the string as a sequence of bytes. + */ +struct ReverseImpl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + for (size_t j = prev_offset; j < offsets[i] - 1; ++j) + res_data[j] = data[offsets[i] + prev_offset - 2 - j]; + res_data[offsets[i] - 1] = 0; + prev_offset = offsets[i]; + } + } + + static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data) + { + res_data.resize(data.size()); + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + for (size_t j = i * n; j < (i + 1) * n; ++j) + res_data[j] = data[(i * 2 + 1) * n - j - 1]; + } +}; + + +class FunctionReverse : public IFunction +{ +public: + static constexpr auto name = "reverse"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isInjective(const Block &) override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0]) + && !isArray(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnFixedString::create(col->getN()); + ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +/// Also works with arrays. +class FunctionBuilderReverse : public FunctionBuilderImpl +{ +public: + static constexpr auto name = "reverse"; + static FunctionBuilderPtr create(const Context & context) { return std::make_shared(context); } + + FunctionBuilderReverse(const Context & context) : context(context) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + +protected: + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (isArray(arguments.at(0).type)) + return FunctionFactory::instance().get("arrayReverse", context)->build(arguments); + else + return std::make_shared( + FunctionReverse::create(context), + ext::map(arguments, [](const auto & elem) { return elem.type; }), + return_type); + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments.at(0); + } + +private: + const Context & context; +}; + + +void registerFunctionReverse(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/reverseUTF8.cpp b/dbms/src/Functions/reverseUTF8.cpp new file mode 100644 index 00000000000..756a959141c --- /dev/null +++ b/dbms/src/Functions/reverseUTF8.cpp @@ -0,0 +1,81 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +/** Reverse the sequence of code points in a UTF-8 encoded string. + * The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols. + * If the string is not encoded in UTF-8, then the behavior is undefined. + */ +struct ReverseUTF8Impl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + ColumnString::Offset j = prev_offset; + while (j < offsets[i] - 1) + { + if (data[j] < 0xBF) + { + res_data[offsets[i] + prev_offset - 2 - j] = data[j]; + j += 1; + } + else if (data[j] < 0xE0) + { + memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 1], &data[j], 2); + j += 2; + } + else if (data[j] < 0xF0) + { + memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 2], &data[j], 3); + j += 3; + } + else + { + res_data[offsets[i] + prev_offset - 2 - j] = data[j]; + j += 1; + } + } + + res_data[offsets[i] - 1] = 0; + prev_offset = offsets[i]; + } + } + + static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) + { + throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +struct NameReverseUTF8 +{ + static constexpr auto name = "reverseUTF8"; +}; +using FunctionReverseUTF8 = FunctionStringToString; + +void registerFunctionReverseUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/rowNumberInAllBlocks.cpp b/dbms/src/Functions/rowNumberInAllBlocks.cpp new file mode 100644 index 00000000000..0ee2ece13d1 --- /dev/null +++ b/dbms/src/Functions/rowNumberInAllBlocks.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Incremental number of row within all blocks passed to this function. */ +class FunctionRowNumberInAllBlocks : public IFunction +{ +private: + std::atomic rows{0}; + +public: + static constexpr auto name = "rowNumberInAllBlocks"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + size_t current_row_number = rows.fetch_add(input_rows_count); + + auto column = ColumnUInt64::create(); + auto & data = column->getData(); + data.resize(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) + data[i] = current_row_number + i; + + block.getByPosition(result).column = std::move(column); + } +}; + + +void registerFunctionRowNumberInAllBlocks(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/rowNumberInBlock.cpp b/dbms/src/Functions/rowNumberInBlock.cpp new file mode 100644 index 00000000000..05ae8add35c --- /dev/null +++ b/dbms/src/Functions/rowNumberInBlock.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionRowNumberInBlock : public IFunction +{ +public: + static constexpr auto name = "rowNumberInBlock"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + auto column = ColumnUInt64::create(); + auto & data = column->getData(); + data.resize(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) + data[i] = i; + + block.getByPosition(result).column = std::move(column); + } +}; + +void registerFunctionRowNumberInBlock(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/runningAccumulate.cpp b/dbms/src/Functions/runningAccumulate.cpp new file mode 100644 index 00000000000..0434c90120d --- /dev/null +++ b/dbms/src/Functions/runningAccumulate.cpp @@ -0,0 +1,107 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values, + * are the result of the accumulation of these states for a set of block lines, from the first to the current line. + * + * Quite unusual function. + * Takes state of aggregate function (example runningAccumulate(uniqState(UserID))), + * and for each row of block, return result of aggregate function on merge of states of all previous rows and current row. + * + * So, result of function depends on partition of data to blocks and on order of data in block. + */ +class FunctionRunningAccumulate : public IFunction +{ +public: + static constexpr auto name = "runningAccumulate"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); + if (!type) + throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnAggregateFunction * column_with_states + = typeid_cast(&*block.getByPosition(arguments.at(0)).column); + if (!column_with_states) + throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + + " of first argument of function " + + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction(); + const IAggregateFunction & agg_func = *aggregate_function_ptr; + + AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData()); + agg_func.create(place.data()); + SCOPE_EXIT(agg_func.destroy(place.data())); + + std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; + + auto result_column_ptr = agg_func.getReturnType()->createColumn(); + IColumn & result_column = *result_column_ptr; + result_column.reserve(column_with_states->size()); + + const auto & states = column_with_states->getData(); + for (const auto & state_to_add : states) + { + /// Will pass empty arena if agg_func does not allocate memory in arena + agg_func.merge(place.data(), state_to_add, arena.get()); + agg_func.insertResultInto(place.data(), result_column); + } + + block.getByPosition(result).column = std::move(result_column_ptr); + } +}; + + +void registerFunctionRunningAccumulate(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/runningDifference.cpp b/dbms/src/Functions/runningDifference.cpp new file mode 100644 index 00000000000..07acaf6522b --- /dev/null +++ b/dbms/src/Functions/runningDifference.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionRunningDifference(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/runningDifference.h b/dbms/src/Functions/runningDifference.h new file mode 100644 index 00000000000..4059d6f3ada --- /dev/null +++ b/dbms/src/Functions/runningDifference.h @@ -0,0 +1,194 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +struct FunctionRunningDifferenceName; + +template <> +struct FunctionRunningDifferenceName +{ + static constexpr auto name = "runningDifference"; +}; + +template <> +struct FunctionRunningDifferenceName +{ + static constexpr auto name = "runningDifferenceStartingWithFirstValue"; +}; + +/** Calculate difference of consecutive values in block. + * So, result of function depends on partition of data to blocks and on order of data in block. + */ +template +class FunctionRunningDifferenceImpl : public IFunction +{ +private: + /// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented. + + template + static void process(const PaddedPODArray & src, PaddedPODArray & dst, const NullMap * null_map) + { + size_t size = src.size(); + dst.resize(size); + + if (size == 0) + return; + + /// It is possible to SIMD optimize this loop. By no need for that in practice. + + Src prev; + bool has_prev_value = false; + + for (size_t i = 0; i < size; ++i) + { + if (null_map && (*null_map)[i]) + continue; + + if (!has_prev_value) + { + dst[i] = is_first_line_zero ? 0 : src[i]; + prev = src[i]; + has_prev_value = true; + } + else + { + auto cur = src[i]; + dst[i] = static_cast(cur) - prev; + prev = cur; + } + } + } + + /// Result type is same as result of subtraction of argument types. + template + using DstFieldType = typename NumberTraits::ResultOfSubtraction::Type; + + /// Call polymorphic lambda with tag argument of concrete field type of src_type. + template + void dispatchForSourceType(const IDataType & src_type, F && f) const + { + WhichDataType which(src_type); + + if (which.isUInt8()) + f(UInt8()); + else if (which.isUInt16()) + f(UInt16()); + else if (which.isUInt32()) + f(UInt32()); + else if (which.isUInt64()) + f(UInt64()); + else if (which.isInt8()) + f(Int8()); + else if (which.isInt16()) + f(Int16()); + else if (which.isInt32()) + f(Int32()); + else if (which.isInt64()) + f(Int64()); + else if (which.isFloat32()) + f(Float32()); + else if (which.isFloat64()) + f(Float64()); + else if (which.isDate()) + f(DataTypeDate::FieldType()); + else if (which.isDateTime()) + f(DataTypeDateTime::FieldType()); + else + throw Exception("Argument for function " + getName() + " must have numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +public: + static constexpr auto name = FunctionRunningDifferenceName::name; + + static FunctionPtr create(const Context &) + { + return std::make_shared>(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + DataTypePtr res; + dispatchForSourceType(*removeNullable(arguments[0]), [&](auto field_type_tag) + { + res = std::make_shared>>(); + }); + + if (arguments[0]->isNullable()) + res = makeNullable(res); + + return res; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto & src = block.getByPosition(arguments.at(0)); + const auto & res_type = block.getByPosition(result).type; + + /// When column is constant, its difference is zero. + if (src.column->isColumnConst()) + { + block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto res_column = removeNullable(res_type)->createColumn(); + auto * src_column = src.column.get(); + ColumnPtr null_map_column = nullptr; + const NullMap * null_map = nullptr; + if (auto * nullable_column = checkAndGetColumn(src_column)) + { + src_column = &nullable_column->getNestedColumn(); + null_map_column = nullable_column->getNullMapColumnPtr(); + null_map = &nullable_column->getNullMapData(); + } + + dispatchForSourceType(*removeNullable(src.type), [&](auto field_type_tag) + { + using SrcFieldType = decltype(field_type_tag); + + process(static_cast &>(*src_column).getData(), + static_cast> &>(*res_column).getData(), null_map); + }); + + if (null_map_column) + block.getByPosition(result).column = ColumnNullable::create(std::move(res_column), null_map_column); + else + block.getByPosition(result).column = std::move(res_column); + } +}; + +} diff --git a/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp b/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp new file mode 100644 index 00000000000..4419e413da7 --- /dev/null +++ b/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionRunningDifferenceStartingWithFirstValue(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/sleep.cpp b/dbms/src/Functions/sleep.cpp new file mode 100644 index 00000000000..ad1c6680363 --- /dev/null +++ b/dbms/src/Functions/sleep.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionSleep(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/sleep.h b/dbms/src/Functions/sleep.h new file mode 100644 index 00000000000..1dfdbf3b5f5 --- /dev/null +++ b/dbms/src/Functions/sleep.h @@ -0,0 +1,99 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_SLOW; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + +/** sleep(seconds) - the specified number of seconds sleeps each block. + */ + +enum class FunctionSleepVariant +{ + PerBlock, + PerRow +}; + +template +class FunctionSleep : public IFunction +{ +public: + static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; + static FunctionPtr create(const Context &) + { + return std::make_shared>(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + /// Do not sleep during query analysis. + bool isSuitableForConstantFolding() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + WhichDataType which(arguments[0]); + + if (!which.isFloat() + && !which.isNativeUInt()) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const IColumn * col = block.getByPosition(arguments[0]).column.get(); + + if (!col->isColumnConst()) + throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); + + Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), static_cast(*col).getField()); + + if (seconds < 0) + throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS); + + size_t size = col->size(); + + /// We do not sleep if the block is empty. + if (size > 0) + { + unsigned useconds = seconds * (variant == FunctionSleepVariant::PerBlock ? 1 : size) * 1e6; + + /// When sleeping, the query cannot be cancelled. For abitily to cancel query, we limit sleep time. + if (useconds > 3000000) /// The choice is arbitary + throw Exception("The maximum sleep time is 3000000 microseconds. Requested: " + toString(useconds), ErrorCodes::TOO_SLOW); + + ::usleep(useconds); + } + + /// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each block. + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, UInt64(0))->convertToFullColumnIfConst(); + } +}; + +} diff --git a/dbms/src/Functions/sleepEachRow.cpp b/dbms/src/Functions/sleepEachRow.cpp new file mode 100644 index 00000000000..c1c983e850e --- /dev/null +++ b/dbms/src/Functions/sleepEachRow.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionSleepEachRow(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/startsWith.cpp b/dbms/src/Functions/startsWith.cpp new file mode 100644 index 00000000000..8e9a06cd17c --- /dev/null +++ b/dbms/src/Functions/startsWith.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionStartsWith = FunctionStartsEndsWith; + +void registerFunctionStartsWith(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/substring.cpp b/dbms/src/Functions/substring.cpp new file mode 100644 index 00000000000..e0a6d707243 --- /dev/null +++ b/dbms/src/Functions/substring.cpp @@ -0,0 +1,172 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ZERO_ARRAY_OR_TUPLE_INDEX; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +class FunctionSubstring : public IFunction +{ +public: + static constexpr auto name = "substring"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + 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 + { + size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isNumber(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + + " of second argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (number_of_arguments == 3 && !isNumber(arguments[2])) + throw Exception("Illegal type " + arguments[2]->getName() + + " of second argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + template + void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length, + const ColumnConst * column_start_const, const ColumnConst * column_length_const, + Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source, + size_t input_rows_count) + { + auto col_res = ColumnString::create(); + + if (!column_length) + { + if (column_start_const) + { + if (start_value > 0) + sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1); + else if (start_value < 0) + sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value); + else + throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + } + else + sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); + } + else + { + if (column_start_const && column_length_const) + { + if (start_value > 0) + sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value); + else if (start_value < 0) + sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value); + else + throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + } + else + sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length); + } + + block.getByPosition(result).column = std::move(col_res); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t number_of_arguments = arguments.size(); + + ColumnPtr column_string = block.getByPosition(arguments[0]).column; + ColumnPtr column_start = block.getByPosition(arguments[1]).column; + ColumnPtr column_length; + + if (number_of_arguments == 3) + column_length = block.getByPosition(arguments[2]).column; + + const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); + const ColumnConst * column_length_const = nullptr; + + if (number_of_arguments == 3) + column_length_const = checkAndGetColumn(column_length.get()); + + Int64 start_value = 0; + Int64 length_value = 0; + + if (column_start_const) + { + start_value = column_start_const->getInt(0); + } + if (column_length_const) + { + length_value = column_length_const->getInt(0); + if (length_value < 0) + throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, StringSource(*col), input_rows_count); + else if (const ColumnFixedString * col = checkAndGetColumn(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, FixedStringSource(*col), input_rows_count); + else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, ConstSource(*col), input_rows_count); + else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, ConstSource(*col), input_rows_count); + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +void registerFunctionSubstring(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/substringUTF8.cpp b/dbms/src/Functions/substringUTF8.cpp new file mode 100644 index 00000000000..aa28a541923 --- /dev/null +++ b/dbms/src/Functions/substringUTF8.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; +} + + +/** If the string is encoded in UTF-8, then it selects a substring of code points in it. + * Otherwise, the behavior is undefined. + */ +struct SubstringUTF8Impl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + size_t start, + size_t length, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.reserve(data.size()); + size_t size = offsets.size(); + res_offsets.resize(size); + + ColumnString::Offset prev_offset = 0; + ColumnString::Offset res_offset = 0; + for (size_t i = 0; i < size; ++i) + { + ColumnString::Offset j = prev_offset; + ColumnString::Offset pos = 1; + ColumnString::Offset bytes_start = 0; + ColumnString::Offset bytes_length = 0; + while (j < offsets[i] - 1) + { + if (pos == start) + bytes_start = j - prev_offset + 1; + + if (data[j] < 0xBF) + j += 1; + else if (data[j] < 0xE0) + j += 2; + else if (data[j] < 0xF0) + j += 3; + else + j += 1; + + if (pos >= start && pos < start + length) + bytes_length = j - prev_offset + 1 - bytes_start; + else if (pos >= start + length) + break; + + ++pos; + } + + if (bytes_start == 0) + { + res_data.resize(res_data.size() + 1); + res_data[res_offset] = 0; + ++res_offset; + } + else + { + size_t bytes_to_copy = std::min(offsets[i] - prev_offset - bytes_start, bytes_length); + res_data.resize(res_data.size() + bytes_to_copy + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + bytes_start - 1], bytes_to_copy); + res_offset += bytes_to_copy + 1; + res_data[res_offset - 1] = 0; + } + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + } +}; + + +class FunctionSubstringUTF8 : public IFunction +{ +public: + static constexpr auto name = "substringUTF8"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 3; + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isNumber(arguments[1]) || !isNumber(arguments[2])) + throw Exception("Illegal type " + (isNumber(arguments[1]) ? arguments[2]->getName() : arguments[1]->getName()) + + " of argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnPtr column_start = block.getByPosition(arguments[1]).column; + const ColumnPtr column_length = block.getByPosition(arguments[2]).column; + + if (!column_start->isColumnConst() || !column_length->isColumnConst()) + throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); + + Field start_field = (*block.getByPosition(arguments[1]).column)[0]; + Field length_field = (*block.getByPosition(arguments[2]).column)[0]; + + if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64) + throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type."); + + UInt64 start = start_field.get(); + UInt64 length = length_field.get(); + + if (start == 0) + throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + /// Otherwise may lead to overflow and pass bounds check inside inner loop. + if (start >= 0x8000000000000000ULL || length >= 0x8000000000000000ULL) + throw Exception("Too large values of 2nd or 3rd argument provided for function substring.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + { + auto col_res = ColumnString::create(); + SubstringUTF8Impl::vector(col->getChars(), col->getOffsets(), start, length, col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +void registerFunctionSubstringUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/throwIf.cpp b/dbms/src/Functions/throwIf.cpp new file mode 100644 index 00000000000..a6d7504414b --- /dev/null +++ b/dbms/src/Functions/throwIf.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO; +} + + +/// Throw an exception if the argument is non zero. +class FunctionThrowIf : public IFunction +{ +public: + static constexpr auto name = "throwIf"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(arguments.front())) + throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto in = block.getByPosition(arguments.front()).column.get(); + + if ( !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result)) + throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + + template + bool execute(Block & block, const IColumn * in_untyped, const size_t result) + { + if (const auto in = checkAndGetColumn>(in_untyped)) + { + const auto & in_data = in->getData(); + if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) + throw Exception("Value passed to 'throwIf' function is non zero", ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO); + + /// We return non constant to avoid constant folding. + block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0); + return true; + } + + return false; + } +}; + + +void registerFunctionThrowIf(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/timezone.cpp b/dbms/src/Functions/timezone.cpp new file mode 100644 index 00000000000..3b3a71c1545 --- /dev/null +++ b/dbms/src/Functions/timezone.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +/** Returns the server time zone. + */ +class FunctionTimeZone : public IFunction +{ +public: + static constexpr auto name = "timezone"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone()); + } +}; + + +void registerFunctionTimeZone(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toColumnTypeName.cpp b/dbms/src/Functions/toColumnTypeName.cpp new file mode 100644 index 00000000000..35d332d082f --- /dev/null +++ b/dbms/src/Functions/toColumnTypeName.cpp @@ -0,0 +1,49 @@ +#include +#include +#include + + +namespace DB +{ + +/// Returns name of IColumn instance. +class FunctionToColumnTypeName : public IFunction +{ +public: + static constexpr auto name = "toColumnTypeName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); + } +}; + + +void registerFunctionToColumnTypeName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toLowCardinality.cpp b/dbms/src/Functions/toLowCardinality.cpp new file mode 100644 index 00000000000..046eb717114 --- /dev/null +++ b/dbms/src/Functions/toLowCardinality.cpp @@ -0,0 +1,58 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionToLowCardinality: public IFunction +{ +public: + static constexpr auto name = "toLowCardinality"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments[0]->withDictionary()) + return arguments[0]; + + return std::make_shared(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + + if (arg.type->withDictionary()) + res.column = arg.column; + else + { + auto column = res.type->createColumn(); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); + res.column = std::move(column); + } + } +}; + + +void registerFunctionToLowCardinality(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toNullable.cpp b/dbms/src/Functions/toNullable.cpp new file mode 100644 index 00000000000..c3709bd85c5 --- /dev/null +++ b/dbms/src/Functions/toNullable.cpp @@ -0,0 +1,49 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// If value is not Nullable or NULL, wraps it to Nullable. +class FunctionToNullable : public IFunction +{ +public: + static constexpr auto name = "toNullable"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return makeNullable(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + block.getByPosition(result).column = makeNullable(block.getByPosition(arguments[0]).column); + } +}; + + +void registerFunctionToNullable(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp new file mode 100644 index 00000000000..560fdaf8dd4 --- /dev/null +++ b/dbms/src/Functions/toTypeName.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + +/** toTypeName(x) - get the type name + * Returns name of IDataType instance (name of data type). + */ +class FunctionToTypeName : public IFunction +{ +public: + static constexpr auto name = "toTypeName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + /// Execute the function on the block. + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).type->getName()); + } +}; + + +void registerFunctionToTypeName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/tuple.cpp b/dbms/src/Functions/tuple.cpp new file mode 100644 index 00000000000..884ec880ffb --- /dev/null +++ b/dbms/src/Functions/tuple.cpp @@ -0,0 +1,86 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +/** tuple(x, y, ...) is a function that allows you to group several columns + * tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple. + */ + +class FunctionTuple : public IFunction +{ +public: + static constexpr auto name = "tuple"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override + { + return true; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isInjective(const Block &) override + { + return true; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() < 1) + throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared(arguments); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + size_t tuple_size = arguments.size(); + Columns tuple_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + { + tuple_columns[i] = block.getByPosition(arguments[i]).column; + + /** If tuple is mixed of constant and not constant columns, + * convert all to non-constant columns, + * because many places in code expect all non-constant columns in non-constant tuple. + */ + if (ColumnPtr converted = tuple_columns[i]->convertToFullColumnIfConst()) + tuple_columns[i] = converted; + } + block.getByPosition(result).column = ColumnTuple::create(tuple_columns); + } +}; + +void registerFunctionTuple(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/FunctionsTuple.cpp b/dbms/src/Functions/tupleElement.cpp similarity index 68% rename from dbms/src/Functions/FunctionsTuple.cpp rename to dbms/src/Functions/tupleElement.cpp index 9df022549bd..99bcc300259 100644 --- a/dbms/src/Functions/FunctionsTuple.cpp +++ b/dbms/src/Functions/tupleElement.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include @@ -17,76 +16,11 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_INDEX; } -/** tuple(x, y, ...) is a function that allows you to group several columns - * tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple. - */ - -class FunctionTuple : public IFunction -{ -public: - static constexpr auto name = "tuple"; - - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isVariadic() const override - { - return true; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isInjective(const Block &) override - { - return true; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() < 1) - throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return std::make_shared(arguments); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - size_t tuple_size = arguments.size(); - Columns tuple_columns(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - { - tuple_columns[i] = block.getByPosition(arguments[i]).column; - - /** If tuple is mixed of constant and not constant columns, - * convert all to non-constant columns, - * because many places in code expect all non-constant columns in non-constant tuple. - */ - if (ColumnPtr converted = tuple_columns[i]->convertToFullColumnIfConst()) - tuple_columns[i] = converted; - } - block.getByPosition(result).column = ColumnTuple::create(tuple_columns); - } -}; - - /** Extract element of tuple by constant index or name. The operation is essentially free. * Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples. */ @@ -200,9 +134,8 @@ private: }; -void registerFunctionsTuple(FunctionFactory & factory) +void registerFunctionTupleElement(FunctionFactory & factory) { - factory.registerFunction(); factory.registerFunction(); } diff --git a/dbms/src/Functions/upper.cpp b/dbms/src/Functions/upper.cpp new file mode 100644 index 00000000000..9a884ccedd8 --- /dev/null +++ b/dbms/src/Functions/upper.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameUpper +{ + static constexpr auto name = "upper"; +}; +using FunctionUpper = FunctionStringToString, NameUpper>; + +void registerFunctionUpper(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/upperUTF8.cpp b/dbms/src/Functions/upperUTF8.cpp new file mode 100644 index 00000000000..048eb7b60c9 --- /dev/null +++ b/dbms/src/Functions/upperUTF8.cpp @@ -0,0 +1,23 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct NameUpperUTF8 +{ + static constexpr auto name = "upperUTF8"; +}; + +using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; + +void registerFunctionUpperUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/uptime.cpp b/dbms/src/Functions/uptime.cpp new file mode 100644 index 00000000000..1f54223eebb --- /dev/null +++ b/dbms/src/Functions/uptime.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** Returns server uptime in seconds. + */ +class FunctionUptime : public IFunction +{ +public: + static constexpr auto name = "uptime"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getUptimeSeconds()); + } + + explicit FunctionUptime(time_t uptime_) : uptime(uptime_) + { + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt32().createColumnConst(input_rows_count, static_cast(uptime)); + } + +private: + time_t uptime; +}; + + +void registerFunctionUptime(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/version.cpp b/dbms/src/Functions/version.cpp new file mode 100644 index 00000000000..3467788a96c --- /dev/null +++ b/dbms/src/Functions/version.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** version() - returns the current version as a string. + */ +class FunctionVersion : public IFunction +{ +public: + static constexpr auto name = "version"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, String(VERSION_STRING)); + } +}; + + +void registerFunctionVersion(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/visibleWidth.cpp b/dbms/src/Functions/visibleWidth.cpp new file mode 100644 index 00000000000..2380bf9a993 --- /dev/null +++ b/dbms/src/Functions/visibleWidth.cpp @@ -0,0 +1,79 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** visibleWidth(x) - calculates the approximate width when outputting the value in a text form to the console. + * In fact it calculate the number of Unicode code points. + * It does not support zero width and full width characters, combining characters, etc. + */ +class FunctionVisibleWidth : public IFunction +{ +public: + static constexpr auto name = "visibleWidth"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool useDefaultImplementationForNulls() const override { return false; } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + /// Execute the function on the block. + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto & src = block.getByPosition(arguments[0]); + size_t size = input_rows_count; + + auto res_col = ColumnUInt64::create(size); + auto & res_data = static_cast(*res_col).getData(); + + /// For simplicity reasons, function is implemented by serializing into temporary buffer. + + String tmp; + FormatSettings format_settings; + for (size_t i = 0; i < size; ++i) + { + { + WriteBufferFromString out(tmp); + src.type->serializeText(*src.column, i, out, format_settings); + } + + res_data[i] = UTF8::countCodePoints(reinterpret_cast(tmp.data()), tmp.size()); + } + + block.getByPosition(result).column = std::move(res_col); + } +}; + + +void registerFunctionVisibleWidth(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index f4874a748bd..53be736c34c 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -473,10 +473,10 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() } /// If single string key - will use hash table with references to it. Strings itself are stored separately in Arena. - if (params.keys_size == 1 && types_removed_nullable[0]->isString()) + if (params.keys_size == 1 && isString(types_removed_nullable[0])) return AggregatedDataVariants::Type::key_string; - if (params.keys_size == 1 && types_removed_nullable[0]->isFixedString()) + if (params.keys_size == 1 && isFixedString(types_removed_nullable[0])) return AggregatedDataVariants::Type::key_fixed_string; return AggregatedDataVariants::Type::serialized; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 942a5a331b3..97f38c627e0 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -60,7 +60,8 @@ Names ExpressionAction::getNeededColumns() const } -ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & function_, +ExpressionAction ExpressionAction::applyFunction( + const FunctionBuilderPtr & function_, const std::vector & argument_names_, std::string result_name_, const std::string & row_projection_column) @@ -86,9 +87,10 @@ ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & func return a; } -ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_, - const std::string & row_projection_column, - bool is_row_projection_complementary) +ExpressionAction ExpressionAction::addColumn( + const ColumnWithTypeAndName & added_column_, + const std::string & row_projection_column, + bool is_row_projection_complementary) { ExpressionAction a; a.type = ADD_COLUMN; @@ -826,7 +828,7 @@ void ExpressionActions::finalize(const Names & output_columns) /// This has to be done before removing redundant actions and inserting REMOVE_COLUMNs /// because inlining may change dependency sets. if (settings.compile_expressions) - compileFunctions(actions, output_columns, sample_block, compilation_cache); + compileFunctions(actions, output_columns, sample_block, compilation_cache, settings.min_count_to_compile); #endif /// Which columns are needed to perform actions from the current to the last. @@ -1139,7 +1141,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh /// It is not important to calculate the hash of individual strings or their concatenation -size_t ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const +UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const { SipHash hash; hash.update(action.type); @@ -1192,7 +1194,9 @@ size_t ExpressionAction::ActionHash::operator()(const ExpressionAction & action) case ADD_ALIASES: break; } - return hash.get64(); + UInt128 result; + hash.get128(result.low, result.high); + return result; } bool ExpressionAction::operator==(const ExpressionAction & other) const diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 9f6d908a117..8d0fbe17cfe 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -129,7 +129,7 @@ public: struct ActionHash { - size_t operator()(const ExpressionAction & action) const; + UInt128 operator()(const ExpressionAction & action) const; }; private: @@ -233,6 +233,20 @@ public: const Settings & getSettings() const { return settings; } + + struct ActionsHash + { + UInt128 operator()(const ExpressionActions::Actions & actions) const + { + SipHash hash; + for (const ExpressionAction & act : actions) + hash.update(ExpressionAction::ActionHash{}(act)); + UInt128 result; + hash.get128(result.low, result.high); + return result; + } + }; + private: NamesAndTypesList input_columns; Actions actions; @@ -252,18 +266,6 @@ private: using ExpressionActionsPtr = std::shared_ptr; -struct ActionsHash -{ - size_t operator()(const ExpressionActions::Actions & actions) const - { - SipHash hash; - for (const ExpressionAction & act : actions) - hash.update(ExpressionAction::ActionHash{}(act)); - return hash.get64(); - } -}; - - /** The sequence of transformations over the block. * It is assumed that the result of each step is fed to the input of the next step. diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 7d28c5e89a2..d5528ad45a5 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -905,18 +905,21 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels) /// Set unique aliases for all subqueries. This is needed, because content of subqueries could change after recursive analysis, /// and auto-generated column names could become incorrect. - size_t subquery_index = 1; - while (true) + if (subquery->alias.empty()) { - alias = "_subquery" + toString(subquery_index); - if (!aliases.count("_subquery" + toString(subquery_index))) - break; - ++subquery_index; - } + size_t subquery_index = 1; + while (true) + { + alias = "_subquery" + toString(subquery_index); + if (!aliases.count("_subquery" + toString(subquery_index))) + break; + ++subquery_index; + } - subquery->setAlias(alias); - subquery->prefer_alias_to_column_name = true; - aliases[alias] = ast; + subquery->setAlias(alias); + subquery->prefer_alias_to_column_name = true; + aliases[alias] = ast; + } } } @@ -2082,12 +2085,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, if (functionIsInOrGlobalInOperator(node->name)) { + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). + getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, projection_manipulator); + if (!no_subqueries) { - /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, - projection_manipulator); - /// Transform tuple or subquery into a set. makeSet(node, actions_stack.getSampleBlock()); } @@ -2096,14 +2098,13 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, if (!only_consts) { /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We insert an arbitrary column of the correct type. - ColumnWithTypeAndName fake_column; - fake_column.name = projection_manipulator->getColumnName(getColumnName()); - fake_column.type = std::make_shared(); - fake_column.column = fake_column.type->createColumn(); - actions_stack.addAction(ExpressionAction::addColumn(fake_column, projection_manipulator->getProjectionSourceColumn(), false)); - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack, - projection_manipulator); + /// Do not subquery and create sets. We treat "IN" as "ignore" function. + + actions_stack.addAction(ExpressionAction::applyFunction( + FunctionFactory::instance().get("ignore", context), + { node->arguments->children.at(0)->getColumnName() }, + projection_manipulator->getColumnName(getColumnName()), + projection_manipulator->getProjectionSourceColumn())); } return; } @@ -2668,13 +2669,17 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty return true; } -bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types) +bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression) { assertSelect(); if (!select_query->prewhere_expression) return false; + Names required_sample_columns; + if (sampling_expression) + required_sample_columns = ExpressionAnalyzer(sampling_expression, context, storage).getRequiredSourceColumns(); + initChain(chain, source_columns); auto & step = chain.getLastStep(); getRootActions(select_query->prewhere_expression, only_types, false, step.actions); @@ -2690,6 +2695,18 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); + /// Add required columns for sample expression to required output in order not to remove them after + /// prewhere execution because sampling is executed after prewhere. + /// TODO: add sampling execution to common chain. + for (const auto & column : required_sample_columns) + { + if (required_source_columns.count(column)) + { + step.required_output.push_back(column); + step.can_remove_required_output.push_back(true); + } + } + auto names = step.actions->getSampleBlock().getNames(); NameSet name_set(names.begin(), names.end()); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 5283d3f1157..86c5cb8ce5a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -142,7 +142,8 @@ public: bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types); bool appendJoin(ExpressionActionsChain & chain, bool only_types); /// remove_filter is set in ExpressionActionsChain::finalize(); - bool appendPrewhere(ExpressionActionsChain & chain, bool only_types); + /// sampling_expression is needed if sampling is used in order to not remove columns are used in it. + bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index ef8b0e410ef..d30ecc7c65a 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -566,12 +566,12 @@ LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType } -static bool isCompilable(llvm::IRBuilderBase & builder, const IFunctionBase & function) +static bool isCompilable(const IFunctionBase & function) { - if (!toNativeType(builder, function.getReturnType())) + if (!canBeNativeType(*function.getReturnType())) return false; for (const auto & type : function.getArgumentTypes()) - if (!toNativeType(builder, type)) + if (!canBeNativeType(*type)) return false; return function.isCompilable(); } @@ -598,21 +598,8 @@ size_t CompiledExpressionCache::weight() const #endif } -void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache) +std::vector>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns) { - struct LLVMTargetInitializer - { - LLVMTargetInitializer() - { - llvm::InitializeNativeTarget(); - llvm::InitializeNativeTargetAsmPrinter(); - llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); - } - }; - - static LLVMTargetInitializer initializer; - - auto context = std::make_shared(); /// an empty optional is a poisoned value prohibiting the column's producer from being removed /// (which it could be, if it was inlined into every dependent function). std::unordered_map>> current_dependents; @@ -656,7 +643,7 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output case ExpressionAction::APPLY_FUNCTION: { dependents[i] = current_dependents[actions[i].result_name]; - const bool compilable = isCompilable(context->builder, *actions[i].function); + const bool compilable = isCompilable(*actions[i].function); for (const auto & name : actions[i].argument_names) { if (compilable) @@ -668,11 +655,34 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output } } } + return dependents; +} +void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache, size_t min_count_to_compile) +{ + static std::unordered_map counter; + static std::mutex mutex; + + struct LLVMTargetInitializer + { + LLVMTargetInitializer() + { + llvm::InitializeNativeTarget(); + llvm::InitializeNativeTargetAsmPrinter(); + llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); + } + }; + + static LLVMTargetInitializer initializer; + + + auto dependents = getActionsDependents(actions, output_columns); + /// Initialize context as late as possible and only if needed + std::shared_ptr context; std::vector fused(actions.size()); for (size_t i = 0; i < actions.size(); ++i) { - if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(context->builder, *actions[i].function)) + if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function)) continue; fused[i].push_back(actions[i]); @@ -682,18 +692,31 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output if (fused[i].size() == 1) continue; + + auto hash_key = ExpressionActions::ActionsHash{}(fused[i]); + { + std::lock_guard lock(mutex); + if (counter[hash_key]++ < min_count_to_compile) + continue; + } std::shared_ptr fn; if (compilation_cache) { - bool success; - auto set_func = [&fused, i, context, &sample_block] () { return std::make_shared(fused[i], context, sample_block); }; - Stopwatch watch; - std::tie(fn, success) = compilation_cache->getOrSet(fused[i], set_func); - if (success) + fn = compilation_cache->get(hash_key); + if (!fn) + { + if (!context) + context = std::make_shared(); + Stopwatch watch; + fn = std::make_shared(fused[i], context, sample_block); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); + compilation_cache->set(hash_key, fn); + } } else { + if (!context) + context = std::make_shared(); Stopwatch watch; fn = std::make_shared(fused[i], context, sample_block); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); @@ -711,7 +734,8 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end()); } - context->finalize(); + if (context) + context->finalize(); } } diff --git a/dbms/src/Interpreters/ExpressionJIT.h b/dbms/src/Interpreters/ExpressionJIT.h index fdcb0d04b93..dc7c8e2c053 100644 --- a/dbms/src/Interpreters/ExpressionJIT.h +++ b/dbms/src/Interpreters/ExpressionJIT.h @@ -61,10 +61,11 @@ public: /** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion. * We have to do so, because we don't known real memory consumption of generated LLVM code for every function. */ -class CompiledExpressionCache : public LRUCache, LLVMFunction, ActionsHash> +class CompiledExpressionCache : public LRUCache { private: - using Base = LRUCache, LLVMFunction, ActionsHash>; + using Base = LRUCache; + public: using Base::Base; @@ -73,7 +74,7 @@ public: /// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable /// function's result are as arguments to other compilable functions, inline it and leave the now-redundant action as-is. -void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache); +void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache, size_t min_count_to_compile); } diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 51bfd4eb788..ba0a3a9e5fd 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -7,7 +7,7 @@ namespace DB namespace { - const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings { }; + const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings {}; const ExternalLoaderConfigSettings & getExternalDictionariesConfigSettings() { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index c4c38c6f73a..da39e756255 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -370,7 +370,7 @@ void InterpreterCreateQuery::checkSupportedTypes(const ColumnsDescription & colu + "Set setting allow_experimental_low_cardinality_type = 1 in order to allow it."; throw Exception(message, ErrorCodes::ILLEGAL_COLUMN); } - if (!allow_decimal && column.type && isDecimal(column.type.get())) + if (!allow_decimal && column.type && isDecimal(column.type)) { String message = "Cannot create table with column '" + column.name + "' which type is '" + column.type->getName() + "'. Set setting allow_experimental_decimal_type = 1 in order to allow it."; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 86164ef2704..ea7913eb2c5 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -101,7 +101,7 @@ BlockIO InterpreterInsertQuery::execute() /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - if (!(context.getSettingsRef().insert_distributed_sync && table->getName() == "Distributed")) + if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote())) { out = std::make_shared( out, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 01d3c28bedf..3049c77523e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -47,6 +47,7 @@ #include #include #include +#include namespace DB @@ -63,6 +64,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int PARAMETER_OUT_OF_BOUND; + extern const int ARGUMENT_OUT_OF_BOUND; } InterpreterSelectQuery::InterpreterSelectQuery( @@ -279,7 +281,6 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() return pipeline.streams; } - InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run) { AnalysisResult res; @@ -305,7 +306,27 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression chain.finalize(); if (has_prewhere) - res.prewhere_info->remove_prewhere_column = chain.steps.at(0).can_remove_required_output.at(0); + { + const ExpressionActionsChain::Step & step = chain.steps.at(0); + res.prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0); + + Names columns_to_remove_after_sampling; + for (size_t i = 1; i < step.required_output.size(); ++i) + { + if (step.can_remove_required_output[i]) + columns_to_remove_after_sampling.push_back(step.required_output[i]); + } + + if (!columns_to_remove_after_sampling.empty()) + { + auto columns = res.prewhere_info->prewhere_actions->getSampleBlock().getNamesAndTypesList(); + ExpressionActionsPtr actions = std::make_shared(columns, context); + for (const auto & column : columns_to_remove_after_sampling) + actions->add(ExpressionAction::removeColumn(column)); + + res.prewhere_info->after_sampling_actions = std::move(actions); + } + } if (has_where) res.remove_where_filter = chain.steps.at(where_step_num).can_remove_required_output.at(0); @@ -317,7 +338,8 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression { ExpressionActionsChain chain(context); - if (query_analyzer->appendPrewhere(chain, !res.first_stage)) + ASTPtr sampling_expression = storage && query.sample_size() ? storage->getSamplingExpression() : nullptr; + if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression)) { has_prewhere = true; @@ -445,8 +467,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (expressions.prewhere_info) pipeline.streams.back() = std::make_shared( pipeline.streams.back(), expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column - ); + expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); } else { diff --git a/dbms/src/Interpreters/PartLog.cpp b/dbms/src/Interpreters/PartLog.cpp index e6a4c1f7a8a..539768123ee 100644 --- a/dbms/src/Interpreters/PartLog.cpp +++ b/dbms/src/Interpreters/PartLog.cpp @@ -18,14 +18,14 @@ namespace DB Block PartLogElement::createBlock() { auto event_type_datatype = std::make_shared( - DataTypeEnum8::Values{ + DataTypeEnum8::Values + { {"NEW_PART", static_cast(NEW_PART)}, {"MERGE_PARTS", static_cast(MERGE_PARTS)}, {"DOWNLOAD_PART", static_cast(DOWNLOAD_PART)}, {"REMOVE_PART", static_cast(REMOVE_PART)}, {"MUTATE_PART", static_cast(MUTATE_PART)}, - } - ); + }); return { diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 22e688f150b..8373f0d0694 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -194,7 +194,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type) } } } - else if (type.isStringOrFixedString()) + else if (isStringOrFixedString(type)) { if (src.getType() == Field::Types::String) return src; diff --git a/dbms/src/Parsers/ASTSubquery.cpp b/dbms/src/Parsers/ASTSubquery.cpp index b0d45cd26f1..55ea89e3f07 100644 --- a/dbms/src/Parsers/ASTSubquery.cpp +++ b/dbms/src/Parsers/ASTSubquery.cpp @@ -8,13 +8,17 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const { /// This is a hack. We use alias, if available, because otherwise tree could change during analysis. if (!alias.empty()) + { writeString(alias, ostr); - - Hash hash = getTreeHash(); - writeCString("__subquery_", ostr); - writeText(hash.first, ostr); - ostr.write('_'); - writeText(hash.second, ostr); + } + else + { + Hash hash = getTreeHash(); + writeCString("__subquery_", ostr); + writeText(hash.first, ostr); + ostr.write('_'); + writeText(hash.second, ostr); + } } void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index cbf69a18a77..b3c5075bc94 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -343,6 +343,9 @@ public: /// Returns data path if storage supports it, empty string otherwise. virtual String getDataPath() const { return {}; } + /// Returns sampling expression for storage or nullptr if there is no. + virtual ASTPtr getSamplingExpression() const { return nullptr; } + protected: using ITableDeclaration::ITableDeclaration; using std::enable_shared_from_this::shared_from_this; diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index a7fe74ffa35..ca2df2e6235 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -708,7 +708,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST - || (key_expr_type->isNumber() && const_type->isNumber()); /// Numbers are accurately compared without cast. + || (isNumber(key_expr_type) && isNumber(const_type)); /// Numbers are accurately compared without cast. if (!cast_not_needed) castValueToType(key_expr_type, const_value, const_type, node); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index c90c37fb9b4..82edb7bafba 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -211,6 +211,16 @@ void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const block.insert({ column, std::make_shared(), virt_column_name}); } + else if (virt_column_name == "_partition_id") + { + ColumnPtr column; + if (rows) + column = DataTypeString().createColumnConst(rows, task->data_part->info.partition_id)->convertToFullColumnIfConst(); + else + column = DataTypeString().createColumn(); + + block.insert({ column, std::make_shared(), virt_column_name}); + } } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp index 8a3fccc5e43..01548d0971d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp @@ -77,8 +77,8 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream( } } - injectVirtualColumns(header); executePrewhereActions(header, prewhere_info); + injectVirtualColumns(header); ordered_names = getHeader().getNames(); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 102378861fe..2e51cfe09e1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -333,6 +333,8 @@ public: return NameAndTypePair("_part", std::make_shared()); if (column_name == "_part_index") return NameAndTypePair("_part_index", std::make_shared()); + if (column_name == "_partition_id") + return NameAndTypePair("_partition_id", std::make_shared()); if (column_name == "_sample_factor") return NameAndTypePair("_sample_factor", std::make_shared()); @@ -344,6 +346,7 @@ public: return getColumns().hasPhysical(column_name) || column_name == "_part" || column_name == "_part_index" + || column_name == "_partition_id" || column_name == "_sample_factor"; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b39f36807ad..3ae37c291d5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -176,6 +176,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( { virt_column_names.push_back(name); } + else if (name == "_partition_id") + { + virt_column_names.push_back(name); + } else if (name == "_sample_factor") { sample_factor_column_queried = true; @@ -593,6 +597,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( stream = std::make_shared>( stream, std::make_shared(), used_sample_factor, "_sample_factor"); + if (query_info.prewhere_info && query_info.prewhere_info->after_sampling_actions) + for (auto & stream : res) + stream = std::make_shared(stream, query_info.prewhere_info->after_sampling_actions); + return res; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp index f1a23933e65..7cfd8f9d539 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp @@ -42,8 +42,8 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( Block MergeTreeThreadBlockInputStream::getHeader() const { auto res = pool->getHeader(); - injectVirtualColumns(res); executePrewhereActions(res, prewhere_info); + injectVirtualColumns(res); return res; } diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index a6e40e4c27d..d875e0cc7ee 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -21,10 +21,12 @@ using PreparedSets = std::unordered_mapwritePrefix(); - block_io.out->write(block); + block_io.out->write(block_to_write); block_io.out->writeSuffix(); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index f561727566d..9b272c45741 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -22,6 +22,8 @@ #include #include #include +#include +#include #include #include @@ -39,10 +41,7 @@ #include #include - #include -#include -#include namespace DB @@ -259,8 +258,8 @@ BlockInputStreams StorageDistributed::read( Block header = materializeBlock(InterpreterSelectQuery(query_info.query, context, Names{}, processed_stage).getSampleBlock()); - ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? - ClusterProxy::SelectStreamFactory( + ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr + ? ClusterProxy::SelectStreamFactory( header, processed_stage, remote_table_function_ptr, context.getExternalTables()) : ClusterProxy::SelectStreamFactory( header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); @@ -346,6 +345,7 @@ namespace {"_table", "String"}, {"_part", "String"}, {"_part_index", "UInt64"}, + {"_partition_id", "String"}, {"_sample_factor", "Float64"}, }; } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index d85de91ab30..27af7e62b6d 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -94,6 +94,8 @@ public: String getDataPath() const override { return full_path; } + ASTPtr getSamplingExpression() const override { return data.sampling_expression; } + private: String path; String database_name; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 69ba9627fc1..779e202fdff 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -193,6 +193,8 @@ public: String getDataPath() const override { return full_path; } + ASTPtr getSamplingExpression() const override { return data.sampling_expression; } + private: /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 17628681847..1ed94bb8077 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -1,11 +1,14 @@ +#include #include #include +#include #include #include #include +#include namespace DB { @@ -13,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; } @@ -31,14 +35,26 @@ StorageView::StorageView( BlockInputStreams StorageView::read( const Names & column_names, - const SelectQueryInfo & /*query_info*/, + const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, const size_t /*max_block_size*/, const unsigned /*num_streams*/) { checkQueryProcessingStage(processed_stage, context); - BlockInputStreams res = InterpreterSelectWithUnionQuery(inner_query, context, column_names).executeWithMultipleStreams(); + + BlockInputStreams res; + + if (context.getSettings().enable_optimize_predicate_expression) + { + replaceTableNameWithSubquery(typeid_cast(query_info.query.get()), inner_query); + auto res_io = InterpreterSelectQuery(query_info.query, context, column_names, processed_stage).execute(); + + res.emplace_back(res_io.in); + return res; + } + + res = InterpreterSelectWithUnionQuery(inner_query, context, column_names).executeWithMultipleStreams(); /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. @@ -48,6 +64,23 @@ BlockInputStreams StorageView::read( return res; } +void StorageView::replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery) +{ + ASTTablesInSelectQueryElement * select_element = static_cast(select_query->tables->children[0].get()); + + if (!select_element->table_expression) + throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); + + ASTTableExpression * table_expression = static_cast(select_element->table_expression.get()); + + if (!table_expression->database_and_table_name) + throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); + + table_expression->database_and_table_name = {}; + table_expression->subquery = std::make_shared(); + table_expression->subquery->children.push_back(subquery->clone()); +} + void registerStorageView(StorageFactory & factory) { diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index 12f666a0648..fed6664b8eb 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -38,6 +39,8 @@ private: String table_name; ASTPtr inner_query; + void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery); + protected: StorageView( const String & table_name_, diff --git a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp index 8fb1f2761de..bf18dec0c86 100644 --- a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp @@ -30,13 +30,13 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons ASTs & args = typeid_cast(*args_func.at(0)).children; if (args.size() != 3) - throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: source, format and structure.", + throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: filename, format and structure.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (size_t i = 0; i < 3; ++i) args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); - std::string source = static_cast(*args[0]).value.safeGet(); + std::string filename = static_cast(*args[0]).value.safeGet(); std::string format = static_cast(*args[1]).value.safeGet(); std::string structure = static_cast(*args[2]).value.safeGet(); @@ -60,7 +60,7 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons } // Create table - StoragePtr storage = getStorage(source, format, sample_block, const_cast(context)); + StoragePtr storage = getStorage(filename, format, sample_block, const_cast(context)); storage->startup(); diff --git a/dbms/tests/performance/jit_query_compilation/small_requests.xml b/dbms/tests/performance/jit_query_compilation/small_requests.xml index 0013c03a1af..b30feade019 100644 --- a/dbms/tests/performance/jit_query_compilation/small_requests.xml +++ b/dbms/tests/performance/jit_query_compilation/small_requests.xml @@ -30,6 +30,8 @@ x3 * 0xc4ceb9fe1a85ec53 AS x4, bitXor(x4, bitShiftRight(x4, 33)) AS x5 SELECT x5, intHash64(number) FROM system.numbers LIMIT 10 + SETTINGS + compile_expressions = 0 WITH @@ -41,7 +43,6 @@ bitXor(x4, bitShiftRight(x4, 33)) AS x5 SELECT x5, intHash64(number) FROM system.numbers LIMIT 10 SETTINGS - compile = 1, compile_expressions = 1 diff --git a/dbms/tests/queries/0_stateless/00396_uuid.reference b/dbms/tests/queries/0_stateless/00396_uuid.reference index 702377653ed..fe92b3684a6 100644 --- a/dbms/tests/queries/0_stateless/00396_uuid.reference +++ b/dbms/tests/queries/0_stateless/00396_uuid.reference @@ -4,3 +4,4 @@ 0123456789ABCDEF0123456789ABCDEF 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef +3f1ed72e-f7fe-4459-9cbe-95fe9298f845 diff --git a/dbms/tests/queries/0_stateless/00396_uuid.sql b/dbms/tests/queries/0_stateless/00396_uuid.sql index 06aa3a64ee7..d671ce844e2 100644 --- a/dbms/tests/queries/0_stateless/00396_uuid.sql +++ b/dbms/tests/queries/0_stateless/00396_uuid.sql @@ -4,3 +4,4 @@ SELECT hex(UUIDStringToNum('01234567-89ab-cdef-0123-456789abcdef')); SELECT hex(UUIDStringToNum(materialize('01234567-89ab-cdef-0123-456789abcdef'))); SELECT '01234567-89ab-cdef-0123-456789abcdef' AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36))); SELECT materialize('01234567-89ab-cdef-0123-456789abcdef') AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36))); +SELECT toString(toUUID('3f1ed72e-f7fe-4459-9cbe-95fe9298f845')); diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index ae742fd40c5..0bea4e90a8e 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -20,5 +20,6 @@ 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 1 2000-01-01 2000-01-01 1 test string 1 1 +2000-01-01 1 test string 1 1 -------Push to having expression, need check.------- -------Compatibility test------- diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 2a58e3d2839..4c803ddaf31 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -1,8 +1,10 @@ SET send_logs_level = 'none'; DROP TABLE IF EXISTS test.test; +DROP TABLE IF EXISTS test.test_view; CREATE TABLE test.test(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192); +CREATE VIEW test.test_view AS SELECT * FROM test.test; INSERT INTO test.test VALUES('2000-01-01', 1, 'test string 1', 1); INSERT INTO test.test VALUES('2000-01-01', 2, 'test string 2', 2); @@ -50,6 +52,9 @@ SELECT * FROM (SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * F -- Optimize predicate expression with join query and qualified SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test) AS b USING date WHERE b.id = 1; +-- Optimize predicate expression with view +SELECT * FROM test.test_view WHERE id = 1; + SELECT '-------Push to having expression, need check.-------'; SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 } @@ -57,3 +62,4 @@ SELECT '-------Compatibility test-------'; SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test) AS b USING date WHERE b.date = toDate('2000-01-01'); -- {serverError 47} DROP TABLE IF EXISTS test.test; +DROP TABLE IF EXISTS test.test_view; diff --git a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql index d7ebd7b0313..f2cf8a06334 100644 --- a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql +++ b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql @@ -1,3 +1,5 @@ +SET send_logs_level = 'none'; + SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 )); SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2)); SELECT sumMerge(y) AS z FROM ( SELECT sumState(x) * 11 AS y FROM ( SELECT 1 AS x UNION ALL SELECT 2 AS x)); @@ -19,4 +21,6 @@ SELECT groupArrayMerge(y * 5) FROM (SELECT groupArrayState(x) AS y FROM (SELECT SELECT groupArrayMerge(2)(y * 5) FROM (SELECT groupArrayState(2)(x) AS y FROM (SELECT 1 AS x)); SELECT groupUniqArrayMerge(y * 5) FROM (SELECT groupUniqArrayState(x) AS y FROM (SELECT 1 AS x)); +SELECT sumMerge(y * a) FROM (SELECT a, sumState(b) AS y FROM test.mult_aggregation GROUP BY a); -- { serverError 44} + DROP TABLE IF EXISTS test.mult_aggregation; diff --git a/dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.sql b/dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.sql index 3e83949c704..dbc4853f840 100644 --- a/dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.sql +++ b/dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.sql @@ -1,9 +1,9 @@ set allow_experimental_low_cardinality_type = 1; select 'NativeBlockInputStream'; -select toTypeName(dict), dict, lowCardinalityIndexes(dict), lowCardinalityKeys(dict) from (select '123_' || toLowCardinality(v) as dict from (select arrayJoin(['a', 'bb', '', 'a', 'ccc', 'a', 'bb', '', 'dddd']) as v)); +select toTypeName(dict), dict, lowCardinalityIndices(dict), lowCardinalityKeys(dict) from (select '123_' || toLowCardinality(v) as dict from (select arrayJoin(['a', 'bb', '', 'a', 'ccc', 'a', 'bb', '', 'dddd']) as v)); select '-'; -select toTypeName(dict), dict, lowCardinalityIndexes(dict), lowCardinalityKeys(dict) from (select '123_' || toLowCardinality(v) as dict from (select arrayJoin(['a', Null, 'bb', '', 'a', Null, 'ccc', 'a', 'bb', '', 'dddd']) as v)); +select toTypeName(dict), dict, lowCardinalityIndices(dict), lowCardinalityKeys(dict) from (select '123_' || toLowCardinality(v) as dict from (select arrayJoin(['a', Null, 'bb', '', 'a', Null, 'ccc', 'a', 'bb', '', 'dddd']) as v)); select 'MergeTree'; diff --git a/dbms/tests/queries/0_stateless/00697_in_subquery_shard.reference b/dbms/tests/queries/0_stateless/00697_in_subquery_shard.reference new file mode 100644 index 00000000000..ce705b56040 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00697_in_subquery_shard.reference @@ -0,0 +1,60 @@ +1 +0 +1 +0 +1 +0 +1 +1 +0 +0 +0 +1 +0 +0 +0 +0 +0 +0 +1 +0 +0 +1 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +90 +1 +1 +0 +0 +0 +1 +0 +0 +0 +0 +0 +0 +1 +0 +0 +1 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +90 diff --git a/dbms/tests/queries/0_stateless/00697_in_subquery_shard.sql b/dbms/tests/queries/0_stateless/00697_in_subquery_shard.sql new file mode 100644 index 00000000000..1e32a587fb3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00697_in_subquery_shard.sql @@ -0,0 +1,25 @@ +SELECT 0 IN 0; +SELECT 0 IN 1; +SELECT 0 IN (SELECT 0); +SELECT 0 IN (SELECT 1); + +SELECT dummy IN (SELECT 0) FROM remote('127.0.0.1', system.one); +SELECT dummy IN (SELECT 1) FROM remote('127.0.0.1', system.one); + +SELECT dummy IN (SELECT 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT dummy IN (SELECT 1) FROM remote('127.0.0.{1,2}', system.one); + +SELECT number IN (SELECT toUInt64(arrayJoin([1, 8]))) FROM remote('127.0.0.{1,2}', numbers(10)); + +SELECT arrayExists(x -> (x IN (SELECT 1)), [1]) FROM remote('127.0.0.{1,2}', system.one); +SELECT sumIf(number, arrayExists(x -> (x IN (SELECT 1)), [1])) FROM remote('127.0.0.{1,2}', numbers(10)); + +SET prefer_localhost_replica = 0; + +SELECT dummy IN (SELECT 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT dummy IN (SELECT 1) FROM remote('127.0.0.{1,2}', system.one); + +SELECT number IN (SELECT toUInt64(arrayJoin([1, 8]))) FROM remote('127.0.0.{1,2}', numbers(10)); + +SELECT arrayExists(x -> (x IN (SELECT 1)), [1]) FROM remote('127.0.0.{1,2}', system.one); +SELECT sumIf(number, arrayExists(x -> (x IN (SELECT 1)), [1])) FROM remote('127.0.0.{1,2}', numbers(10)); diff --git a/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.reference b/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.reference new file mode 100644 index 00000000000..9969db5c4b5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.reference @@ -0,0 +1,6 @@ +4 +7 4 3 +1 +3 +[1,1,2,3] +[1,2,3] diff --git a/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.sql b/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.sql new file mode 100644 index 00000000000..65f42313ccb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00705_aggregate_states_addition.sql @@ -0,0 +1,21 @@ +USE test; +SET send_logs_level = 'none'; +DROP TABLE IF EXISTS add_aggregate; +CREATE TABLE add_aggregate(a UInt32, b UInt32) ENGINE = Memory; + +INSERT INTO add_aggregate VALUES(1, 2); +INSERT INTO add_aggregate VALUES(3, 1); + +SELECT countMerge(x + y) FROM (SELECT countState(a) as x, countState(b) as y from add_aggregate); +SELECT sumMerge(x + y), sumMerge(x), sumMerge(y) FROM (SELECT sumState(a) as x, sumState(b) as y from add_aggregate); +SELECT sumMerge(x) FROM (SELECT sumState(a) + countState(b) as x FROM add_aggregate); -- { serverError 419 } +SELECT sumMerge(x) FROM (SELECT sumState(a) + sumState(toInt32(b)) as x FROM add_aggregate); -- { serverError 419 } + +SELECT minMerge(x) FROM (SELECT minState(a) + minState(b) as x FROM add_aggregate); + +SELECT uniqMerge(x + y) FROM (SELECT uniqState(a) as x, uniqState(b) as y FROM add_aggregate); + +SELECT arraySort(groupArrayMerge(x + y)) FROM (SELECT groupArrayState(a) AS x, groupArrayState(b) as y FROM add_aggregate); +SELECT arraySort(groupUniqArrayMerge(x + y)) FROM (SELECT groupUniqArrayState(a) AS x, groupUniqArrayState(b) as y FROM add_aggregate); + +DROP TABLE IF EXISTS add_aggregate; diff --git a/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.reference b/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.reference new file mode 100644 index 00000000000..8566e252b8f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.reference @@ -0,0 +1,3 @@ +197004 +197007 +197010 diff --git a/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.sql b/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.sql new file mode 100644 index 00000000000..17492a92f33 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00709_virtual_column_partition_id.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test.partition_id; + +CREATE TABLE IF NOT EXISTS test.partition_id (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5); + +INSERT INTO test.partition_id VALUES (100, 1), (200, 2), (300, 3); + +SELECT _partition_id FROM test.partition_id ORDER BY x; + +DROP TABLE IF EXISTS test.partition_id; + diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.reference b/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.reference new file mode 100644 index 00000000000..b261da18d51 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.reference @@ -0,0 +1,2 @@ +1 +0 diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.sql b/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.sql new file mode 100644 index 00000000000..32bfeccf556 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_sampling.sql @@ -0,0 +1,10 @@ +drop table if exists test.tab; +create table test.tab (a UInt32, b UInt32) engine = MergeTree order by b % 2 sample by b % 2; +insert into test.tab values (1, 2), (1, 4); +select a from test.tab sample 1 / 2 prewhere b = 2; +drop table if exists test.tab; + +DROP TABLE IF EXISTS test.sample_prewhere; +CREATE TABLE test.sample_prewhere (CounterID UInt32, UserID UInt64) ENGINE = MergeTree ORDER BY UserID SAMPLE BY UserID; +SELECT count() FROM test.sample_prewhere SAMPLE 1/2 PREWHERE CounterID = 1; +DROP TABLE test.sample_prewhere; diff --git a/dbms/tests/queries/0_stateless/00714_alter_uuid.reference b/dbms/tests/queries/0_stateless/00714_alter_uuid.reference new file mode 100644 index 00000000000..48991600c93 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00714_alter_uuid.reference @@ -0,0 +1,6 @@ +00000000-0000-01f8-9cb8-cb1b82fb3900 00000000-0000-01f8-9cb8-cb1b82fb3900 +00000000-0000-02f8-9cb8-cb1b82fb3900 00000000-0000-02f8-9cb8-cb1b82fb3900 +00000000-0000-03f8-9cb8-cb1b82fb3900 00000000-0000-03f8-9cb8-cb1b82fb3900 +00000000-0000-04f8-9cb8-cb1b82fb3900 00000000-0000-04f8-9cb8-cb1b82fb3900 +00000000-0000-05f8-9cb8-cb1b82fb3900 00000000-0000-06f8-9cb8-cb1b82fb3900 +UUID UUID diff --git a/dbms/tests/queries/0_stateless/00714_alter_uuid.sql b/dbms/tests/queries/0_stateless/00714_alter_uuid.sql new file mode 100644 index 00000000000..116d640785a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00714_alter_uuid.sql @@ -0,0 +1,47 @@ +USE test; + +SELECT '00000000-0000-01f8-9cb8-cb1b82fb3900' AS str, toUUID(str); +SELECT toFixedString('00000000-0000-02f8-9cb8-cb1b82fb3900', 36) AS str, toUUID(str); + +SELECT '00000000-0000-03f8-9cb8-cb1b82fb3900' AS str, CAST(str, 'UUID'); +SELECT toFixedString('00000000-0000-04f8-9cb8-cb1b82fb3900', 36) AS str, CAST(str, 'UUID'); + +DROP TABLE IF EXISTS uuid; +CREATE TABLE IF NOT EXISTS uuid +( + created_at DateTime, + id0 String, + id1 FixedString(36) +) +ENGINE = MergeTree +PARTITION BY toDate(created_at) +ORDER BY (created_at); + +INSERT INTO uuid VALUES ('2018-01-01 01:02:03', '00000000-0000-05f8-9cb8-cb1b82fb3900', '00000000-0000-06f8-9cb8-cb1b82fb3900'); + +ALTER TABLE uuid MODIFY COLUMN id0 UUID; +ALTER TABLE uuid MODIFY COLUMN id1 UUID; + +SELECT id0, id1 FROM uuid; +SELECT toTypeName(id0), toTypeName(id1) FROM uuid; + +DROP TABLE uuid; + +-- with UUID in key + +CREATE TABLE IF NOT EXISTS uuid +( + created_at DateTime, + id0 String, + id1 FixedString(36) +) +ENGINE = MergeTree +PARTITION BY toDate(created_at) +ORDER BY (created_at, id0, id1); + +SET send_logs_level = 'none'; + +ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 44 } +ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 44 } + +DROP TABLE uuid; diff --git a/debian/changelog b/debian/changelog index e04fb161db7..e5d16a2d312 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.12.8) unstable; urgency=low +clickhouse (18.12.11) unstable; urgency=low * Modified source code - -- Fri, 07 Sep 2018 04:43:52 +0300 + -- Mon, 10 Sep 2018 13:44:17 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 96e4c55b5b5..e34ba98b4f9 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.8 +ARG version=18.12.11 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 0fd74646bdf..20e07d5651d 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.8 +ARG version=18.12.11 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 8e050c65d2e..53996fd4d3d 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.8 +ARG version=18.12.11 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \