mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
6368daff63
@ -25,17 +25,10 @@ endif ()
|
||||
# Write compile_commands.json
|
||||
set(CMAKE_EXPORT_COMPILE_COMMANDS 1)
|
||||
|
||||
set(PARALLEL_COMPILE_JOBS "" CACHE STRING "Define the maximum number of concurrent compilation jobs")
|
||||
if (PARALLEL_COMPILE_JOBS)
|
||||
set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool="${PARALLEL_COMPILE_JOBS}")
|
||||
set(CMAKE_JOB_POOL_COMPILE compile_job_pool)
|
||||
endif ()
|
||||
|
||||
set(PARALLEL_LINK_JOBS "" CACHE STRING "Define the maximum number of concurrent link jobs")
|
||||
if (LLVM_PARALLEL_LINK_JOBS)
|
||||
set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS})
|
||||
set(CMAKE_JOB_POOL_LINK link_job_pool)
|
||||
endif ()
|
||||
set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "")
|
||||
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
|
||||
include (cmake/limit_jobs.cmake)
|
||||
|
||||
include (cmake/find_ccache.cmake)
|
||||
|
||||
@ -162,51 +155,8 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fn
|
||||
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}")
|
||||
set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}")
|
||||
|
||||
set(THREADS_PREFER_PTHREAD_FLAG ON)
|
||||
find_package (Threads)
|
||||
|
||||
include (cmake/test_compiler.cmake)
|
||||
|
||||
if (OS_LINUX AND COMPILER_CLANG)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}")
|
||||
|
||||
option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX})
|
||||
set (LIBCXX_PATH "" CACHE STRING "Use custom path for libc++. It should be used for MSan.")
|
||||
|
||||
if (USE_LIBCXX)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning
|
||||
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build.
|
||||
if (MAKE_STATIC_LIBRARIES)
|
||||
execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c)
|
||||
else ()
|
||||
link_libraries (-stdlib=libc++ c++ c++abi)
|
||||
endif ()
|
||||
|
||||
if (LIBCXX_PATH)
|
||||
# include_directories (SYSTEM BEFORE "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1")
|
||||
link_directories ("${LIBCXX_PATH}/lib")
|
||||
endif ()
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
if (USE_LIBCXX)
|
||||
set (STATIC_STDLIB_FLAGS "")
|
||||
else ()
|
||||
set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++")
|
||||
endif ()
|
||||
|
||||
if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
|
||||
|
||||
# Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained.
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
|
||||
endif ()
|
||||
|
||||
if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||
endif ()
|
||||
include (cmake/use_libcxx.cmake)
|
||||
|
||||
if (NOT MAKE_STATIC_LIBRARIES)
|
||||
set(CMAKE_POSITION_INDEPENDENT_CODE ON)
|
||||
|
35
cmake/limit_jobs.cmake
Normal file
35
cmake/limit_jobs.cmake
Normal file
@ -0,0 +1,35 @@
|
||||
# Usage:
|
||||
# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # In megabytes
|
||||
# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
|
||||
# include (cmake/limit_jobs.cmake)
|
||||
|
||||
cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd
|
||||
|
||||
option(PARALLEL_COMPILE_JOBS "Define the maximum number of concurrent compilation jobs" "")
|
||||
if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY)
|
||||
math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/2500) # ~2.5gb max per one compiler
|
||||
if (NOT PARALLEL_COMPILE_JOBS)
|
||||
set (PARALLEL_COMPILE_JOBS 1)
|
||||
endif ()
|
||||
endif ()
|
||||
if (PARALLEL_COMPILE_JOBS)
|
||||
set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool=${PARALLEL_COMPILE_JOBS})
|
||||
set(CMAKE_JOB_POOL_COMPILE compile_job_pool)
|
||||
endif ()
|
||||
|
||||
option(PARALLEL_LINK_JOBS "Define the maximum number of concurrent link jobs" "")
|
||||
if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY)
|
||||
math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/4000) # ~4gb max per one linker
|
||||
if (NOT PARALLEL_LINK_JOBS)
|
||||
set (PARALLEL_LINK_JOBS 1)
|
||||
endif ()
|
||||
endif ()
|
||||
if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS)
|
||||
message(STATUS "Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}")
|
||||
endif ()
|
||||
|
||||
if (LLVM_PARALLEL_LINK_JOBS)
|
||||
set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS})
|
||||
set(CMAKE_JOB_POOL_LINK link_job_pool)
|
||||
endif ()
|
||||
|
49
cmake/use_libcxx.cmake
Normal file
49
cmake/use_libcxx.cmake
Normal file
@ -0,0 +1,49 @@
|
||||
# Uses MAKE_STATIC_LIBRARIES
|
||||
|
||||
|
||||
set(THREADS_PREFER_PTHREAD_FLAG ON)
|
||||
find_package (Threads)
|
||||
|
||||
include (cmake/test_compiler.cmake)
|
||||
include (cmake/arch.cmake)
|
||||
|
||||
if (OS_LINUX AND COMPILER_CLANG)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}")
|
||||
|
||||
option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX})
|
||||
set (LIBCXX_PATH "" CACHE STRING "Use custom path for libc++. It should be used for MSan.")
|
||||
|
||||
if (USE_LIBCXX)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning
|
||||
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build.
|
||||
if (MAKE_STATIC_LIBRARIES)
|
||||
execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c)
|
||||
else ()
|
||||
link_libraries (-stdlib=libc++ c++ c++abi)
|
||||
endif ()
|
||||
|
||||
if (LIBCXX_PATH)
|
||||
# include_directories (SYSTEM BEFORE "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1")
|
||||
link_directories ("${LIBCXX_PATH}/lib")
|
||||
endif ()
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
if (USE_LIBCXX)
|
||||
set (STATIC_STDLIB_FLAGS "")
|
||||
else ()
|
||||
set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++")
|
||||
endif ()
|
||||
|
||||
if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
|
||||
|
||||
# Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained.
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
|
||||
endif ()
|
||||
|
||||
if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||
endif ()
|
@ -27,12 +27,12 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE})
|
||||
endif ()
|
||||
|
||||
if (COPY_HEADERS_COMPILER AND OS_LINUX)
|
||||
add_custom_target (copy-headers ALL env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
|
||||
add_custom_target (copy-headers env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
|
||||
|
||||
if (USE_INTERNAL_LLVM_LIBRARY)
|
||||
set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers")
|
||||
set (CLANG_HEADERS_DEST "${TMP_HEADERS_DIR}/usr/local/lib/clang/${LLVM_VERSION}/include") # original: ${LLVM_LIBRARY_OUTPUT_INTDIR}/clang/${CLANG_VERSION}/include
|
||||
add_custom_target (copy-headers-clang ALL ${CMAKE_COMMAND} -E make_directory ${CLANG_HEADERS_DEST} && ${CMAKE_COMMAND} -E copy_if_different ${CLANG_HEADERS_DIR}/* ${CLANG_HEADERS_DEST} )
|
||||
add_custom_target (copy-headers-clang ${CMAKE_COMMAND} -E make_directory ${CLANG_HEADERS_DEST} && ${CMAKE_COMMAND} -E copy_if_different ${CLANG_HEADERS_DIR}/* ${CLANG_HEADERS_DEST} )
|
||||
add_dependencies (copy-headers copy-headers-clang)
|
||||
endif ()
|
||||
endif ()
|
||||
|
@ -322,17 +322,11 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const
|
||||
|
||||
ColumnPtr ColumnArray::convertToFullColumnIfConst() const
|
||||
{
|
||||
ColumnPtr new_data;
|
||||
|
||||
if (ColumnPtr full_column = getData().convertToFullColumnIfConst())
|
||||
new_data = full_column;
|
||||
else
|
||||
new_data = data;
|
||||
|
||||
return ColumnArray::create(new_data, offsets);
|
||||
/// It is possible to have an array with constant data and non-constant offsets.
|
||||
/// Example is the result of expression: replicate('hello', [1])
|
||||
return ColumnArray::create(data->convertToFullColumnIfConst(), offsets);
|
||||
}
|
||||
|
||||
|
||||
void ColumnArray::getExtremes(Field & min, Field & max) const
|
||||
{
|
||||
min = Array();
|
||||
|
@ -22,8 +22,7 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn
|
||||
: nested_column(std::move(nested_column_)), null_map(std::move(null_map_))
|
||||
{
|
||||
/// ColumnNullable cannot have constant nested column. But constant argument could be passed. Materialize it.
|
||||
if (ColumnPtr nested_column_materialized = getNestedColumn().convertToFullColumnIfConst())
|
||||
nested_column = nested_column_materialized;
|
||||
nested_column = getNestedColumn().convertToFullColumnIfConst();
|
||||
|
||||
if (!getNestedColumn().canBeInsideNullable())
|
||||
throw Exception{getNestedColumn().getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
@ -45,7 +45,7 @@ public:
|
||||
/** If column isn't constant, returns nullptr (or itself).
|
||||
* If column is constant, transforms constant to full column (if column type allows such tranform) and return it.
|
||||
*/
|
||||
virtual Ptr convertToFullColumnIfConst() const { return {}; }
|
||||
virtual Ptr convertToFullColumnIfConst() const { return getPtr(); }
|
||||
|
||||
/// If column isn't ColumnLowCardinality, return itself.
|
||||
/// If column is ColumnLowCardinality, transforms is to full column.
|
||||
|
@ -46,12 +46,7 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
|
||||
/** If there are columns-constants - then we materialize them.
|
||||
* (Since the data type does not know how to serialize / deserialize constants.)
|
||||
*/
|
||||
ColumnPtr full_column;
|
||||
|
||||
if (ColumnPtr converted = column->convertToFullColumnIfConst())
|
||||
full_column = converted;
|
||||
else
|
||||
full_column = column;
|
||||
ColumnPtr full_column = column->convertToFullColumnIfConst();
|
||||
|
||||
IDataType::SerializeBinaryBulkSettings settings;
|
||||
settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; };
|
||||
|
@ -127,10 +127,7 @@ Block TotalsHavingBlockInputStream::readImpl()
|
||||
expression->execute(finalized);
|
||||
|
||||
size_t filter_column_pos = finalized.getPositionByName(filter_column_name);
|
||||
ColumnPtr filter_column_ptr = finalized.safeGetByPosition(filter_column_pos).column;
|
||||
|
||||
if (ColumnPtr materialized = filter_column_ptr->convertToFullColumnIfConst())
|
||||
filter_column_ptr = materialized;
|
||||
ColumnPtr filter_column_ptr = finalized.safeGetByPosition(filter_column_pos).column->convertToFullColumnIfConst();
|
||||
|
||||
FilterDescription filter_description(*filter_column_ptr);
|
||||
|
||||
|
@ -14,9 +14,7 @@ Block materializeBlock(const Block & block)
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
auto & element = res.getByPosition(i);
|
||||
auto & src = element.column;
|
||||
if (ColumnPtr converted = src->convertToFullColumnIfConst())
|
||||
src = converted;
|
||||
element.column = element.column->convertToFullColumnIfConst();
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -1,8 +1,7 @@
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
|
||||
add_headers_and_sources(clickhouse_functions .)
|
||||
add_headers_and_sources(clickhouse_functions ./GatherUtils)
|
||||
add_headers_and_sources(clickhouse_functions ./Conditional)
|
||||
add_headers_and_sources(clickhouse_functions .)
|
||||
|
||||
|
||||
list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp)
|
||||
|
@ -11,13 +11,14 @@
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include "IFunction.h"
|
||||
#include "FunctionHelpers.h"
|
||||
#include "intDiv.h"
|
||||
#include "castTypeToEither.h"
|
||||
#include "FunctionFactory.h"
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Functions/intDiv.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
@ -346,11 +346,8 @@ private:
|
||||
String attr_name = attr_name_col->getValue<String>();
|
||||
|
||||
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
|
||||
ColumnPtr key_col = key_col_with_type.column;
|
||||
|
||||
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
|
||||
if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst())
|
||||
key_col = key_col_materialized;
|
||||
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
|
||||
|
||||
if (checkColumn<ColumnTuple>(key_col.get()))
|
||||
{
|
||||
@ -578,11 +575,8 @@ private:
|
||||
String attr_name = attr_name_col->getValue<String>();
|
||||
|
||||
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
|
||||
ColumnPtr key_col = key_col_with_type.column;
|
||||
|
||||
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
|
||||
if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst())
|
||||
key_col = key_col_materialized;
|
||||
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
|
||||
|
||||
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
@ -813,11 +807,9 @@ private:
|
||||
String attr_name = attr_name_col->getValue<String>();
|
||||
|
||||
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
|
||||
ColumnPtr key_col = key_col_with_type.column;
|
||||
|
||||
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
|
||||
if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst())
|
||||
key_col = key_col_materialized;
|
||||
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
|
||||
|
||||
if (checkColumn<ColumnTuple>(key_col.get()))
|
||||
{
|
||||
@ -1079,11 +1071,9 @@ private:
|
||||
String attr_name = attr_name_col->getValue<String>();
|
||||
|
||||
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
|
||||
ColumnPtr key_col = key_col_with_type.column;
|
||||
|
||||
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
|
||||
if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst())
|
||||
key_col = key_col_materialized;
|
||||
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
|
||||
|
||||
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
@ -1691,7 +1681,7 @@ static const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & c
|
||||
}
|
||||
}
|
||||
|
||||
const auto full_column = column.isColumnConst() ? column.convertToFullColumnIfConst() : column.getPtr();
|
||||
const auto full_column = column.convertToFullColumnIfConst();
|
||||
|
||||
// With type conversion and const columns we need to use backup storage here
|
||||
const auto size = full_column->size();
|
||||
|
@ -227,8 +227,7 @@ protected:
|
||||
bool executeOperationTyped(const IColumn * in_untyped, PaddedPODArray<OutputType> & dst, const IColumn * centroids_array_untyped)
|
||||
{
|
||||
const auto maybe_const = in_untyped->convertToFullColumnIfConst();
|
||||
if (maybe_const)
|
||||
in_untyped = maybe_const.get();
|
||||
in_untyped = maybe_const.get();
|
||||
|
||||
const auto in_vector = checkAndGetColumn<ColumnVector<InputType>>(in_untyped);
|
||||
if (in_vector)
|
||||
|
@ -157,10 +157,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum
|
||||
if (!result_null_map_column)
|
||||
return makeNullable(src);
|
||||
|
||||
if (src_not_nullable->isColumnConst())
|
||||
return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column);
|
||||
else
|
||||
return ColumnNullable::create(src_not_nullable, result_null_map_column);
|
||||
return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column);
|
||||
}
|
||||
|
||||
|
||||
@ -431,9 +428,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
|
||||
|
||||
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows(), dry_run);
|
||||
|
||||
auto & keys = block_without_low_cardinality.safeGetByPosition(result).column;
|
||||
if (auto full_column = keys->convertToFullColumnIfConst())
|
||||
keys = full_column;
|
||||
auto keys = block_without_low_cardinality.safeGetByPosition(result).column->convertToFullColumnIfConst();
|
||||
|
||||
auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType());
|
||||
ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size());
|
||||
|
@ -69,8 +69,7 @@ public:
|
||||
if (!arg.type->equals(*elem_type))
|
||||
preprocessed_column = castColumn(arg, elem_type, context);
|
||||
|
||||
if (ColumnPtr materialized_column = preprocessed_column->convertToFullColumnIfConst())
|
||||
preprocessed_column = materialized_column;
|
||||
preprocessed_column = preprocessed_column->convertToFullColumnIfConst();
|
||||
|
||||
columns_holder[i] = std::move(preprocessed_column);
|
||||
columns[i] = columns_holder[i].get();
|
||||
|
@ -61,21 +61,10 @@ private:
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
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);
|
||||
bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
|
||||
bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
|
||||
bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
|
||||
bool executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
|
||||
};
|
||||
|
||||
|
||||
@ -83,14 +72,14 @@ template <typename Derived>
|
||||
void FunctionArrayEnumerateExtended<Derived>::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());
|
||||
size_t num_arguments = arguments.size();
|
||||
ColumnRawPtrs data_columns(num_arguments);
|
||||
|
||||
bool has_nullable_columns = false;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
Columns array_holders;
|
||||
ColumnPtr offsets_column;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
@ -100,101 +89,84 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const C
|
||||
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<ColumnArray>(array_ptr.get());
|
||||
array_holders.emplace_back(const_array->convertToFullColumn());
|
||||
array = checkAndGetColumn<ColumnArray>(array_holders.back().get());
|
||||
}
|
||||
|
||||
const ColumnArray::Offsets & offsets_i = array->getOffsets();
|
||||
if (i == 0)
|
||||
{
|
||||
offsets = &offsets_i;
|
||||
offsets_column = array->getOffsetsPtr();
|
||||
}
|
||||
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);
|
||||
data_columns[i] = array_data;
|
||||
}
|
||||
|
||||
size_t num_columns = data_columns.size();
|
||||
ColumnRawPtrs original_data_columns(num_columns);
|
||||
ColumnRawPtrs null_maps(num_columns);
|
||||
const NullMap * null_map = nullptr;
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
original_data_columns[i] = data_columns[i];
|
||||
|
||||
if (data_columns[i]->isColumnNullable())
|
||||
{
|
||||
has_nullable_columns = true;
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(*data_columns[i]);
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
null_maps[i] = &nullable_col.getNullMapColumn();
|
||||
|
||||
if (num_arguments == 1)
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
|
||||
null_map = &nullable_col.getNullMapData();
|
||||
break;
|
||||
}
|
||||
else
|
||||
null_maps[i] = nullptr;
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = checkAndGetColumn<ColumnArray>(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 (num_arguments == 1)
|
||||
{
|
||||
if (!(executeNumber<UInt8>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt16>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt64>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int8>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int16>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int64>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float64>(first_array, first_null_map, res_values)
|
||||
|| executeString (first_array, first_null_map, res_values)))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int8>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int16>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeString(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
execute128bit(*offsets, data_columns, res_values)
|
||||
|| executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), first_array->getOffsetsPtr());
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column);
|
||||
}
|
||||
|
||||
|
||||
template <typename Derived>
|
||||
template <typename T>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeNumber(
|
||||
const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * 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<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(inner_col);
|
||||
if (!nested)
|
||||
const ColumnVector<T> * data_concrete = checkAndGetColumn<ColumnVector<T>>(&data);
|
||||
if (!data_concrete)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container & values = nested->getData();
|
||||
const auto & values = data_concrete->getData();
|
||||
|
||||
using ValuesToIndices = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
@ -207,7 +179,7 @@ bool FunctionArrayEnumerateExtended<Derived>::executeNumber(const ColumnArray *
|
||||
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_map && (*null_map)[j])
|
||||
res_values[j] = ++null_count;
|
||||
else
|
||||
res_values[j] = ++indices[values[j]];
|
||||
@ -226,7 +198,7 @@ bool FunctionArrayEnumerateExtended<Derived>::executeNumber(const ColumnArray *
|
||||
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_map && (*null_map)[j])
|
||||
{
|
||||
if (!null_index)
|
||||
null_index = ++rank;
|
||||
@ -247,32 +219,17 @@ bool FunctionArrayEnumerateExtended<Derived>::executeNumber(const ColumnArray *
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeString(
|
||||
const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * 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<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnString * nested = checkAndGetColumn<ColumnString>(inner_col);
|
||||
if (!nested)
|
||||
const ColumnString * values = checkAndGetColumn<ColumnString>(&data);
|
||||
if (!values)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
size_t prev_off = 0;
|
||||
using ValuesToIndices = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
ValuesToIndices indices;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
{
|
||||
@ -284,10 +241,10 @@ bool FunctionArrayEnumerateExtended<Derived>::executeString(const ColumnArray *
|
||||
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_map && (*null_map)[j])
|
||||
res_values[j] = ++null_count;
|
||||
else
|
||||
res_values[j] = ++indices[nested->getDataAt(j)];
|
||||
res_values[j] = ++indices[values->getDataAt(j)];
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
@ -303,7 +260,7 @@ bool FunctionArrayEnumerateExtended<Derived>::executeString(const ColumnArray *
|
||||
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_map && (*null_map)[j])
|
||||
{
|
||||
if (!null_index)
|
||||
null_index = ++rank;
|
||||
@ -311,7 +268,7 @@ bool FunctionArrayEnumerateExtended<Derived>::executeString(const ColumnArray *
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & idx = indices[nested->getDataAt(j)];
|
||||
auto & idx = indices[values->getDataAt(j)];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
@ -327,9 +284,7 @@ template <typename Derived>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns)
|
||||
ColumnUInt32::Container & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
@ -342,8 +297,6 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
key_sizes[j] = columns[j]->sizeOfValueIfFixed();
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
if (has_nullable_columns)
|
||||
keys_bytes += std::tuple_size<KeysNullMap<UInt128>>::value;
|
||||
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
@ -361,29 +314,7 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
indices.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*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<UInt128>(j, count, columns, key_sizes, bitmap)];
|
||||
}
|
||||
else
|
||||
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes)];
|
||||
}
|
||||
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes)];
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
@ -397,35 +328,10 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
size_t rank = 0;
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*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<UInt128>(j, count, columns, key_sizes, bitmap)];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto &idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];;
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
auto &idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];;
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
@ -435,7 +341,7 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
void FunctionArrayEnumerateExtended<Derived>::executeHashed(
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values)
|
||||
@ -479,6 +385,8 @@ void FunctionArrayEnumerateExtended<Derived>::executeHashed(
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -838,15 +838,9 @@ private:
|
||||
null_map_data, nullptr);
|
||||
else
|
||||
{
|
||||
/// If item_arg is tuple and have constants.
|
||||
if (ColumnPtr materialized_tuple = item_arg.convertToFullColumnIfConst())
|
||||
ArrayIndexGenericImpl<IndexConv, false>::vector(
|
||||
col_nested, col_array->getOffsets(), *materialized_tuple, col_res->getData(),
|
||||
null_map_data, null_map_item);
|
||||
else
|
||||
ArrayIndexGenericImpl<IndexConv, false>::vector(
|
||||
col_nested, col_array->getOffsets(), item_arg, col_res->getData(),
|
||||
null_map_data, null_map_item);
|
||||
ArrayIndexGenericImpl<IndexConv, false>::vector(
|
||||
col_nested, col_array->getOffsets(), *item_arg.convertToFullColumnIfConst(), col_res->getData(),
|
||||
null_map_data, null_map_item);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
|
@ -23,9 +23,7 @@ struct ArrayMapImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
|
||||
{
|
||||
return mapped->isColumnConst()
|
||||
? ColumnArray::create(mapped->convertToFullColumnIfConst(), array.getOffsetsPtr())
|
||||
: ColumnArray::create(mapped, array.getOffsetsPtr());
|
||||
return ColumnArray::create(mapped->convertToFullColumnIfConst(), array.getOffsetsPtr());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -63,37 +63,23 @@ private:
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
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);
|
||||
bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
|
||||
bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
|
||||
bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
|
||||
bool 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());
|
||||
size_t num_arguments = arguments.size();
|
||||
ColumnRawPtrs data_columns(num_arguments);
|
||||
|
||||
bool has_nullable_columns = false;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
Columns array_holders;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
@ -101,14 +87,12 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|
||||
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(),
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = static_cast<const ColumnArray *>(array_ptr.get());
|
||||
array_holders.emplace_back(const_array->convertToFullColumn());
|
||||
array = checkAndGetColumn<ColumnArray>(array_holders.back().get());
|
||||
}
|
||||
|
||||
array_columns[i] = array_ptr;
|
||||
|
||||
const ColumnArray::Offsets & offsets_i = array->getOffsets();
|
||||
if (i == 0)
|
||||
offsets = &offsets_i;
|
||||
@ -116,78 +100,65 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|
||||
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<const ColumnNullable &>(*data_columns[i]);
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
null_maps[i] = &nullable_col.getNullMapColumn();
|
||||
}
|
||||
else
|
||||
null_maps[i] = nullptr;
|
||||
auto * array_data = &array->getData();
|
||||
data_columns[i] = array_data;
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = static_cast<const ColumnArray *>(array_columns[0].get());
|
||||
const IColumn * first_null_map = null_maps[0];
|
||||
auto res = ColumnUInt32::create();
|
||||
const NullMap * null_map = nullptr;
|
||||
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
if (data_columns[i]->isColumnNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(*data_columns[i]);
|
||||
|
||||
if (num_arguments == 1)
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
|
||||
null_map = &nullable_col.getNullMapData();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
auto res = ColumnUInt32::create();
|
||||
ColumnUInt32::Container & res_values = res->getData();
|
||||
res_values.resize(offsets->size());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
if (num_arguments == 1)
|
||||
{
|
||||
if (!(executeNumber<UInt8>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt16>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt64>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int8>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int16>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int64>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float32>(first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float64>(first_array, first_null_map, res_values)
|
||||
|| executeString(first_array, first_null_map, res_values)))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int8>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int16>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Int64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeString(*offsets, *data_columns[0], null_map, res_values)
|
||||
|| executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
execute128bit(*offsets, data_columns, res_values)
|
||||
|| executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * 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<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(inner_col);
|
||||
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(&data);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container & values = nested->getData();
|
||||
const auto & values = nested->getData();
|
||||
|
||||
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
@ -197,7 +168,7 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn *
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
if (null_map && (*null_map)[j])
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(values[j]);
|
||||
@ -209,31 +180,15 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn *
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
bool FunctionArrayUniq::executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * 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<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnString * nested = checkAndGetColumn<ColumnString>(inner_col);
|
||||
const ColumnString * nested = checkAndGetColumn<ColumnString>(&data);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
@ -243,7 +198,7 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn *
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
if (null_map && (*null_map)[j])
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(nested->getDataAt(j));
|
||||
@ -259,9 +214,7 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn *
|
||||
bool FunctionArrayUniq::execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns)
|
||||
ColumnUInt32::Container & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
@ -274,8 +227,6 @@ bool FunctionArrayUniq::execute128bit(
|
||||
key_sizes[j] = columns[j]->sizeOfValueIfFixed();
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
if (has_nullable_columns)
|
||||
keys_bytes += std::tuple_size<KeysNullMap<UInt128>>::value;
|
||||
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
@ -283,19 +234,6 @@ bool FunctionArrayUniq::execute128bit(
|
||||
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
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)
|
||||
@ -303,29 +241,7 @@ bool FunctionArrayUniq::execute128bit(
|
||||
set.clear();
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (ColumnArray::Offset i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*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<UInt128>(j, count, columns, key_sizes, bitmap));
|
||||
}
|
||||
else
|
||||
set.insert(packFixed<UInt128>(j, count, columns, key_sizes));
|
||||
}
|
||||
set.insert(packFixed<UInt128>(j, count, columns, key_sizes));
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
@ -334,7 +250,7 @@ bool FunctionArrayUniq::execute128bit(
|
||||
return true;
|
||||
}
|
||||
|
||||
void FunctionArrayUniq::executeHashed(
|
||||
bool FunctionArrayUniq::executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values)
|
||||
@ -356,6 +272,8 @@ void FunctionArrayUniq::executeHashed(
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
|
@ -638,9 +638,7 @@ private:
|
||||
|
||||
static ColumnPtr materializeColumnIfConst(const ColumnPtr & column)
|
||||
{
|
||||
if (ColumnPtr res = column->convertToFullColumnIfConst())
|
||||
return res;
|
||||
return column;
|
||||
return column->convertToFullColumnIfConst();
|
||||
}
|
||||
|
||||
static ColumnPtr makeNullableColumnIfNot(const ColumnPtr & column)
|
||||
|
@ -34,11 +34,7 @@ public:
|
||||
|
||||
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;
|
||||
block.getByPosition(result).column = block.getByPosition(arguments[0]).column->convertToFullColumnIfConst();
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -92,7 +92,9 @@ private:
|
||||
{
|
||||
#if __SSE4_2__
|
||||
/// skip whitespace from left in blocks of up to 16 characters
|
||||
constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT;
|
||||
|
||||
/// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate
|
||||
enum { left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT };
|
||||
while (mask == bytes_sse && chars_to_trim_left < size_sse)
|
||||
{
|
||||
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data + chars_to_trim_left));
|
||||
@ -110,7 +112,9 @@ private:
|
||||
const auto trim_right_size = size - chars_to_trim_left;
|
||||
#if __SSE4_2__
|
||||
/// try to skip whitespace from right in blocks of up to 16 characters
|
||||
constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT;
|
||||
|
||||
/// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate
|
||||
enum { right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT };
|
||||
const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse);
|
||||
while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse)
|
||||
{
|
||||
|
@ -65,14 +65,11 @@ public:
|
||||
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;
|
||||
* convert all to non-constant columns,
|
||||
* because many places in code expect all non-constant columns in non-constant tuple.
|
||||
*/
|
||||
tuple_columns[i] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst();
|
||||
}
|
||||
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
|
||||
}
|
||||
|
@ -772,13 +772,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.safeGetByPosition(params.keys[i]).column.get();
|
||||
|
||||
if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.push_back(converted);
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
}
|
||||
materialized_columns.push_back(block.safeGetByPosition(params.keys[i]).column->convertToFullColumnIfConst());
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
|
||||
if (const auto * low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
|
||||
{
|
||||
@ -797,13 +792,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
|
||||
{
|
||||
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
|
||||
{
|
||||
aggregate_columns[i][j] = block.safeGetByPosition(params.aggregates[i].arguments[j]).column.get();
|
||||
|
||||
if (ColumnPtr converted = aggregate_columns[i][j]->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.push_back(converted);
|
||||
aggregate_columns[i][j] = materialized_columns.back().get();
|
||||
}
|
||||
materialized_columns.push_back(block.safeGetByPosition(params.aggregates[i].arguments[j]).column->convertToFullColumnIfConst());
|
||||
aggregate_columns[i][j] = materialized_columns.back().get();
|
||||
|
||||
if (auto * col_low_cardinality = typeid_cast<const ColumnLowCardinality *>(aggregate_columns[i][j]))
|
||||
{
|
||||
|
@ -375,10 +375,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
if (array_joined_columns.empty())
|
||||
throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column;
|
||||
if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst())
|
||||
any_array_ptr = converted;
|
||||
|
||||
ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column->convertToFullColumnIfConst();
|
||||
const ColumnArray * any_array = typeid_cast<const ColumnArray *>(&*any_array_ptr);
|
||||
if (!any_array)
|
||||
throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH);
|
||||
@ -416,10 +413,10 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
|
||||
Block tmp_block{src_col, column_of_max_length, {{}, src_col.type, {}}};
|
||||
function_arrayResize->build({src_col, column_of_max_length})->execute(tmp_block, {0, 1}, 2, rows);
|
||||
any_array_ptr = src_col.column = tmp_block.safeGetByPosition(2).column;
|
||||
src_col.column = tmp_block.safeGetByPosition(2).column;
|
||||
any_array_ptr = src_col.column->convertToFullColumnIfConst();
|
||||
}
|
||||
if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst())
|
||||
any_array_ptr = converted;
|
||||
|
||||
any_array = typeid_cast<const ColumnArray *>(&*any_array_ptr);
|
||||
}
|
||||
else if (array_join_is_left && !unaligned_array_join)
|
||||
@ -434,10 +431,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
non_empty_array_columns[name] = tmp_block.safeGetByPosition(1).column;
|
||||
}
|
||||
|
||||
any_array_ptr = non_empty_array_columns.begin()->second;
|
||||
if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst())
|
||||
any_array_ptr = converted;
|
||||
|
||||
any_array_ptr = non_empty_array_columns.begin()->second->convertToFullColumnIfConst();
|
||||
any_array = &typeid_cast<const ColumnArray &>(*any_array_ptr);
|
||||
}
|
||||
|
||||
@ -452,9 +446,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
throw Exception("ARRAY JOIN of not array: " + current.name, ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
ColumnPtr array_ptr = (array_join_is_left && !unaligned_array_join) ? non_empty_array_columns[current.name] : current.column;
|
||||
|
||||
if (ColumnPtr converted = array_ptr->convertToFullColumnIfConst())
|
||||
array_ptr = converted;
|
||||
array_ptr = array_ptr->convertToFullColumnIfConst();
|
||||
|
||||
const ColumnArray & array = typeid_cast<const ColumnArray &>(*array_ptr);
|
||||
if (!unaligned_array_join && !array.hasEqualOffsets(typeid_cast<const ColumnArray &>(*any_array_ptr)))
|
||||
|
@ -437,14 +437,8 @@ bool Join::insertFromBlock(const Block & block)
|
||||
/// Memoize key columns to work.
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column));
|
||||
materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column->convertToFullColumnIfConst()));
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
|
||||
if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
}
|
||||
}
|
||||
|
||||
/// We will insert to the map only keys, where all components are not NULL.
|
||||
@ -483,11 +477,7 @@ bool Join::insertFromBlock(const Block & block)
|
||||
|
||||
/// Rare case, when joined columns are constant. To avoid code bloat, simply materialize them.
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnPtr col = stored_block->safeGetByPosition(i).column;
|
||||
if (ColumnPtr converted = col->convertToFullColumnIfConst())
|
||||
stored_block->safeGetByPosition(i).column = converted;
|
||||
}
|
||||
stored_block->safeGetByPosition(i).column = stored_block->safeGetByPosition(i).column->convertToFullColumnIfConst();
|
||||
|
||||
/// In case of LEFT and FULL joins, if use_nulls, convert joined columns to Nullable.
|
||||
if (use_nulls && (kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full))
|
||||
@ -685,14 +675,8 @@ void Join::joinBlockImpl(
|
||||
/// Memoize key columns to work with.
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_left[i]).column));
|
||||
materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_left[i]).column->convertToFullColumnIfConst()));
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
|
||||
if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
}
|
||||
}
|
||||
|
||||
/// Keys with NULL value in any column won't join to anything.
|
||||
@ -710,10 +694,7 @@ void Join::joinBlockImpl(
|
||||
{
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
{
|
||||
auto & col = block.getByPosition(i).column;
|
||||
|
||||
if (ColumnPtr converted = col->convertToFullColumnIfConst())
|
||||
col = converted;
|
||||
block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst();
|
||||
|
||||
/// If use_nulls, convert left columns (except keys) to Nullable.
|
||||
if (use_nulls)
|
||||
|
@ -121,15 +121,10 @@ void Set::setHeader(const Block & block)
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns.emplace_back(block.safeGetByPosition(i).column.get());
|
||||
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst());
|
||||
key_columns.emplace_back(materialized_columns.back().get());
|
||||
data_types.emplace_back(block.safeGetByPosition(i).type);
|
||||
|
||||
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
|
||||
/// Convert low cardinality column to full.
|
||||
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(data_types.back().get()))
|
||||
{
|
||||
@ -175,20 +170,8 @@ bool Set::insertFromBlock(const Block & block)
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns.emplace_back(block.safeGetByPosition(i).column.get());
|
||||
|
||||
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
|
||||
/// Convert low cardinality column to full.
|
||||
if (key_columns.back()->lowCardinality())
|
||||
{
|
||||
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality());
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
|
||||
key_columns.emplace_back(materialized_columns.back().get());
|
||||
}
|
||||
|
||||
size_t rows = block.rows();
|
||||
@ -365,18 +348,13 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
|
||||
for (size_t i = 0; i < num_key_columns; ++i)
|
||||
{
|
||||
key_columns.push_back(block.safeGetByPosition(i).column.get());
|
||||
|
||||
if (!removeNullable(data_types[i])->equals(*removeNullable(block.safeGetByPosition(i).type)))
|
||||
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: "
|
||||
+ data_types[i]->getName() + " on the right, " + block.safeGetByPosition(i).type->getName() +
|
||||
" on the left.", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst());
|
||||
key_columns.emplace_back() = materialized_columns.back().get();
|
||||
}
|
||||
|
||||
/// We will check existence in Set only for keys, where all components are not NULL.
|
||||
|
@ -67,8 +67,7 @@ void evaluateMissingDefaults(Block & block,
|
||||
if (copy_block.has(col->name))
|
||||
{
|
||||
auto evaluated_col = copy_block.getByName(col->name);
|
||||
if (ColumnPtr converted = evaluated_col.column->convertToFullColumnIfConst())
|
||||
evaluated_col.column = converted;
|
||||
evaluated_col.column = evaluated_col.column->convertToFullColumnIfConst();
|
||||
|
||||
block.insert(pos, std::move(evaluated_col));
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ void ASTAlterCommand::formatImpl(
|
||||
|
||||
if (type == ASTAlterCommand::ADD_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
col_decl->formatImpl(settings, state, frame);
|
||||
|
||||
/// AFTER
|
||||
@ -64,7 +64,7 @@ void ASTAlterCommand::formatImpl(
|
||||
else if (type == ASTAlterCommand::DROP_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
<< (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
column->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
@ -74,7 +74,7 @@ void ASTAlterCommand::formatImpl(
|
||||
}
|
||||
else if (type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
col_decl->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
|
@ -78,6 +78,10 @@ public:
|
||||
|
||||
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
|
||||
|
||||
bool if_not_exists = false; /// option for ADD_COLUMN
|
||||
|
||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
|
||||
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
|
||||
*/
|
||||
String from;
|
||||
|
@ -36,6 +36,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_partition("PARTITION");
|
||||
|
||||
ParserKeyword s_after("AFTER");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_if_exists("IF EXISTS");
|
||||
ParserKeyword s_from("FROM");
|
||||
ParserKeyword s_in_partition("IN PARTITION");
|
||||
ParserKeyword s_with("WITH");
|
||||
@ -57,6 +59,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_add_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_col_decl.parse(pos, command->col_decl, expected))
|
||||
return false;
|
||||
|
||||
@ -77,6 +82,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_drop_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
@ -85,6 +93,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
@ -190,6 +201,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_modify_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_modify_col_decl.parse(pos, command->col_decl, expected))
|
||||
return false;
|
||||
|
||||
@ -224,6 +238,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_comment_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
|
@ -8,12 +8,12 @@ namespace DB
|
||||
|
||||
/** Query like this:
|
||||
* ALTER TABLE [db.]name [ON CLUSTER cluster]
|
||||
* [ADD COLUMN col_name type [AFTER col_after],]
|
||||
* [DROP COLUMN col_to_drop, ...]
|
||||
* [CLEAR COLUMN col_to_clear [IN PARTITION partition],]
|
||||
* [MODIFY COLUMN col_to_modify type, ...]
|
||||
* [ADD COLUMN [IF NOT EXISTS] col_name type [AFTER col_after],]
|
||||
* [DROP COLUMN [IF EXISTS] col_to_drop, ...]
|
||||
* [CLEAR COLUMN [IF EXISTS] col_to_clear [IN PARTITION partition],]
|
||||
* [MODIFY COLUMN [IF EXISTS] col_to_modify type, ...]
|
||||
* [MODIFY PRIMARY KEY (a, b, c...)]
|
||||
* [COMMENT COLUMN col_name string]
|
||||
* [COMMENT COLUMN [IF EXISTS] col_name string]
|
||||
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
|
||||
* [FETCH PARTITION partition FROM ...]
|
||||
* [FREEZE [PARTITION] [WITH NAME name]]
|
||||
|
@ -52,6 +52,8 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
if (command_ast->column)
|
||||
command.after_column = typeid_cast<const ASTIdentifier &>(*command_ast->column).name;
|
||||
|
||||
command.if_not_exists = command_ast->if_not_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition)
|
||||
@ -62,6 +64,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::DROP_COLUMN;
|
||||
command.column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
|
||||
command.if_exists = command_ast->if_exists;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
@ -88,6 +91,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
const auto & ast_comment = typeid_cast<ASTLiteral &>(*ast_col_decl.comment);
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
}
|
||||
command.if_exists = command_ast->if_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
@ -99,6 +103,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.column_name = ast_identifier.name;
|
||||
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
command.if_exists = command_ast->if_exists;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
@ -300,7 +305,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr & ord
|
||||
auto new_primary_key_ast = primary_key_ast;
|
||||
|
||||
for (const AlterCommand & command : *this)
|
||||
command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast);
|
||||
if (!command.ignore)
|
||||
command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
columns_description = std::move(new_columns_description);
|
||||
order_by_ast = std::move(new_order_by_ast);
|
||||
@ -328,45 +334,61 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
if (command.type == AlterCommand::ADD_COLUMN)
|
||||
{
|
||||
if (std::end(all_columns) != column_it)
|
||||
throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN};
|
||||
{
|
||||
if (command.if_not_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
|
||||
if (std::end(all_columns) == column_it)
|
||||
throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN};
|
||||
{
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
all_columns.erase(column_it);
|
||||
defaults.erase(column_name);
|
||||
if (!command.ignore)
|
||||
{
|
||||
all_columns.erase(column_it);
|
||||
defaults.erase(column_name);
|
||||
}
|
||||
}
|
||||
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared<DataTypeUInt8>());
|
||||
|
||||
if (command.default_expression)
|
||||
if (!command.ignore)
|
||||
{
|
||||
if (command.data_type)
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared<DataTypeUInt8>());
|
||||
|
||||
if (command.default_expression)
|
||||
{
|
||||
const auto & final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto column_type_raw_ptr = command.data_type.get();
|
||||
if (command.data_type)
|
||||
{
|
||||
const auto &final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto column_type_raw_ptr = command.data_type.get();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
|
||||
std::make_shared<ASTLiteral>(column_type_raw_ptr->getName())),
|
||||
final_column_name));
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
|
||||
std::make_shared<ASTLiteral>(column_type_raw_ptr->getName())),
|
||||
final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// no type explicitly specified, will deduce later
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(command.default_expression->clone(), column_name));
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// no type explicitly specified, will deduce later
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(command.default_expression->clone(), column_name));
|
||||
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command);
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -407,8 +429,13 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
}
|
||||
|
||||
if (!found)
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
{
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
@ -416,7 +443,10 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
std::bind(namesEqual, std::cref(command.column_name), std::placeholders::_1));
|
||||
if (column_it == std::end(all_columns))
|
||||
{
|
||||
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -43,15 +43,26 @@ struct AlterCommand
|
||||
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
|
||||
String after_column;
|
||||
|
||||
/// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
bool if_exists;
|
||||
|
||||
/// For ADD_COLUMN
|
||||
bool if_not_exists;
|
||||
|
||||
/// For MODIFY_ORDER_BY
|
||||
ASTPtr order_by;
|
||||
|
||||
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
|
||||
bool ignore = false;
|
||||
|
||||
AlterCommand() = default;
|
||||
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
|
||||
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,
|
||||
const String & after_column = String{}, const String & comment = "") // TODO: разобраться здесь с параметром по умолчанию
|
||||
const String & after_column = String{}, const String & comment = "",
|
||||
const bool if_exists = false, const bool if_not_exists = false) // TODO: разобраться здесь с параметром по умолчанию
|
||||
: type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind},
|
||||
default_expression{default_expression}, comment(comment), after_column{after_column}
|
||||
default_expression{default_expression}, comment(comment), after_column{after_column},
|
||||
if_exists(if_exists), if_not_exists(if_not_exists)
|
||||
{}
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
@ -166,9 +166,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c
|
||||
|
||||
/// Filter the block.
|
||||
String filter_column_name = expression_ast->getColumnName();
|
||||
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column;
|
||||
if (ColumnPtr converted = filter_column->convertToFullColumnIfConst())
|
||||
filter_column = converted;
|
||||
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst();
|
||||
const IColumn::Filter & filter = typeid_cast<const ColumnUInt8 &>(*filter_column).getData();
|
||||
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
|
@ -23,6 +23,13 @@ ALTER TABLE test.alter_test DROP COLUMN NestedColumn.S;
|
||||
|
||||
ALTER TABLE test.alter_test DROP COLUMN AddedNested1.B;
|
||||
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32;
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64);
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String);
|
||||
ALTER TABLE test.alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64;
|
||||
ALTER TABLE test.alter_test DROP COLUMN IF EXISTS ToDrop;
|
||||
ALTER TABLE test.alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment';
|
||||
|
||||
DESC TABLE test.alter_test;
|
||||
|
||||
SELECT * FROM test.alter_test;
|
||||
|
@ -0,0 +1,2 @@
|
||||
[]
|
||||
[1]
|
@ -0,0 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SELECT arrayEnumerateUniq(anyHeavy([]), []);
|
||||
SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 }
|
||||
SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]);
|
@ -3,9 +3,7 @@ SELECT globalNotIn(['"wh'], [NULL]);
|
||||
SELECT globalIn([''], [NULL])
|
||||
SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]);
|
||||
SELECT truncate(895, -16);
|
||||
SELECT arrayEnumerateUniq(anyHeavy([]), []);
|
||||
SELECT notIn([['']], [[NULL]]);
|
||||
SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895);
|
||||
SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', [];
|
||||
SELECT addDays((CAST((96.338) AS DateTime)), -3);
|
||||
SELECT arrayEnumerateDense([], [sequenceCount(NULL)]);
|
||||
|
@ -16,10 +16,12 @@ sh ${CUR_DIR}check-include > $RESULT_FILE 2>&1
|
||||
|
||||
echo Results:
|
||||
echo Top by memory:
|
||||
cat $RESULT_FILE | sort -rk4 | head -n20
|
||||
cat $RESULT_FILE | sort -nrk4 | head -n20
|
||||
|
||||
echo Top by time:
|
||||
cat $RESULT_FILE | sort -rk3 | head -n20
|
||||
cat $RESULT_FILE | sort -nrk3 | head -n20
|
||||
|
||||
echo Top by includes:
|
||||
cat $RESULT_FILE | sort -rk2 | head -n20
|
||||
cat $RESULT_FILE | sort -nrk2 | head -n20
|
||||
|
||||
trap "" EXIT
|
||||
|
Loading…
Reference in New Issue
Block a user