Merge branch 'master' into decimal

This commit is contained in:
chertus 2018-09-06 11:57:46 +03:00
commit 3bd586cad9
74 changed files with 874 additions and 168 deletions

View File

@ -38,7 +38,8 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
endif ()
if (USE_DEBUG_HELPERS)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -include ${ClickHouse_SOURCE_DIR}/libs/libcommon/include/common/iostream_debug_helpers.h")
set (INCLUDE_DEBUG_HELPERS "-include ${ClickHouse_SOURCE_DIR}/libs/libcommon/include/common/iostream_debug_helpers.h")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}")
endif ()
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")

View File

@ -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 2 CACHE STRING "")
set(VERSION_GITHASH d12c1b02bc50119d67db2690c6bc7aeeae9d55ef CACHE STRING "")
set(VERSION_DESCRIBE v18.12.2-testing CACHE STRING "")
set(VERSION_STRING 18.12.2 CACHE STRING "")
set(VERSION_PATCH 5 CACHE STRING "")
set(VERSION_GITHASH d8c528ea3973dbcfb68227fc0eff0feffa399d3d CACHE STRING "")
set(VERSION_DESCRIBE v18.12.5-testing CACHE STRING "")
set(VERSION_STRING 18.12.5 CACHE STRING "")
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -18,8 +18,16 @@ set(TMP_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/${INTERNAL_COMPILER_HEADERS_REL
add_custom_target (make-headers-directory ALL COMMAND ${CMAKE_COMMAND} -E make_directory ${TMP_HEADERS_DIR})
install (DIRECTORY ${TMP_HEADERS_DIR} DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse/${INTERNAL_COMPILER_HEADERS_DIR} COMPONENT clickhouse)
# TODO: fix on macos copy_headers.sh: sed --posix
if (USE_EMBEDDED_COMPILER AND NOT APPLE)
add_custom_target(copy-headers ALL env CLANG=${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-clang BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS clickhouse-clang WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
if (USE_EMBEDDED_COMPILER)
set (COPY_HEADERS_COMPILER "${CMAKE_CURRENT_BINARY_DIR}/../${INTERNAL_COMPILER_EXECUTABLE}")
set (COPY_HEADERS_DEPENDS clickhouse-clang)
elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE})
set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}")
endif ()
if (COPY_HEADERS_COMPILER AND NOT APPLE)
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)
if (USE_INTERNAL_LLVM_LIBRARY)
set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers")

View File

@ -220,7 +220,7 @@ public:
{
char * res = alloc(new_size);
if (old_data)
memcpySmallAllowReadWriteOverflow15(res, old_data, old_size);
memcpy(res, old_data, old_size);
return res;
}
@ -228,7 +228,7 @@ public:
{
char * res = alignedAlloc(new_size, alignment);
if (old_data)
memcpySmallAllowReadWriteOverflow15(res, old_data, old_size);
memcpy(res, old_data, old_size);
return res;
}
@ -236,14 +236,14 @@ public:
const char * insert(const char * data, size_t size)
{
char * res = alloc(size);
memcpySmallAllowReadWriteOverflow15(res, data, size);
memcpy(res, data, size);
return res;
}
const char * alignedInsert(const char * data, size_t size, size_t alignment)
{
char * res = alignedAlloc(size, alignment);
memcpySmallAllowReadWriteOverflow15(res, data, size);
memcpy(res, data, size);
return res;
}

View File

@ -21,6 +21,7 @@ namespace ErrorCodes
thread_local ThreadStatusPtr current_thread = ThreadStatus::create();
thread_local CurrentThread::ThreadScopePtr current_thread_scope = std::make_shared<CurrentThread::ThreadScope>();
void CurrentThread::updatePerformanceCounters()
{
get()->updatePerformanceCounters();

View File

@ -5,6 +5,7 @@
#include <Common/ThreadStatus.h>
namespace ProfileEvents
{
class Counters;
@ -21,10 +22,15 @@ class QueryStatus;
struct Progress;
class InternalTextLogsQueue;
/** Collection of static methods to work with thread-local objects.
* Allows to attach and detach query/process (thread group) to a thread
* (to calculate query-related metrics and to allow to obtain query-related data from a thread).
* Thread will propagate it's metrics to attached query.
*/
class CurrentThread
{
public:
/// Handler to current thread
static ThreadStatusPtr get();

View File

@ -13,6 +13,9 @@ struct taskstats {};
#endif
/** Implement ProfileEvents with statistics about resource consumption of the current thread.
*/
namespace ProfileEvents
{
extern const Event RealTimeMicroseconds;

View File

@ -34,6 +34,14 @@ using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
using InternalTextLogsQueueWeakPtr = std::weak_ptr<InternalTextLogsQueue>;
/** Thread group is a collection of threads dedicated to single task
* (query or other process like background merge).
*
* ProfileEvents (counters) from a thread are propagated to thread group.
*
* Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks).
* Use via CurrentThread::getGroup.
*/
class ThreadGroupStatus
{
public:
@ -60,10 +68,14 @@ public:
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
/** Encapsulates all per-thread info (ProfileEvents, MemoryTracker, query_id, query context, etc.).
* Used inside thread-local variable. See variables in CurrentThread.cpp
*
* This object should be used only via "CurrentThread", see CurrentThread.h
*/
class ThreadStatus : public std::enable_shared_from_this<ThreadStatus>
{
public:
/// Poco's thread number (the same number is used in logs)
UInt32 thread_number = 0;
/// Linux's PID (or TGID) (the same id is shown by ps util)
@ -78,7 +90,6 @@ public:
Progress progress_out;
public:
static ThreadStatusPtr create();
ThreadGroupStatusPtr getThreadGroup() const

View File

@ -421,8 +421,7 @@ private:
void createConcrete(T && x)
{
using JustT = std::decay_t<T>;
JustT * MAY_ALIAS ptr = reinterpret_cast<JustT *>(&storage);
new (ptr) JustT(std::forward<T>(x));
new (&storage) JustT(std::forward<T>(x));
which = TypeToEnum<JustT>::value;
}
@ -492,8 +491,7 @@ private:
void create(const char * data, size_t size)
{
String * MAY_ALIAS ptr = reinterpret_cast<String*>(&storage);
new (ptr) String(data, size);
new (&storage) String(data, size);
which = Types::String;
}

View File

@ -48,12 +48,8 @@ void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_column
return;
}
if (count_positive == count_negative && !last_is_positive)
if (last_is_positive || count_positive != count_negative)
{
/// Input rows exactly cancel out.
return;
}
if (count_positive <= count_negative)
{
++merged_rows;
@ -80,6 +76,7 @@ void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_column
reportIncorrectData();
++count_incorrect_data;
}
}
if (out_row_sources_buf)
out_row_sources_buf->write(

View File

@ -0,0 +1,73 @@
#include <DataStreams/RollupBlockInputStream.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/FilterDescription.h>
#include <Common/typeid_cast.h>
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<const DataTypeAggregateFunction *>(current.type.get());
if (unfinalized_type)
{
current.type = unfinalized_type->getReturnType();
if (current.column)
current.column = typeid_cast<const ColumnAggregateFunction &>(*current.column).convertToValues();
}
}
}
RollupBlockInputStream::RollupBlockInputStream(
const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : aggregator(params_),
keys(params_.keys)
{
children.push_back(input_);
Aggregator::CancellationHook hook = [this]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook);
}
Block RollupBlockInputStream::getHeader() const
{
Block res = children.at(0)->getHeader();
finalize(res);
return res;
}
Block RollupBlockInputStream::readImpl()
{
/** After reading a block from input stream,
* we will subsequently roll it up on next iterations of 'readImpl'
* by zeroing out every column one-by-one and re-merging a block.
*/
if (current_key >= 0)
{
auto & current = rollup_block.getByPosition(keys[current_key]);
current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows());
--current_key;
BlocksList rollup_blocks = { rollup_block };
rollup_block = aggregator.mergeBlocks(rollup_blocks, false);
Block finalized = rollup_block;
finalize(finalized);
return finalized;
}
Block block = children[0]->read();
current_key = keys.size() - 1;
rollup_block = block;
finalize(block);
return block;
}
}

View File

@ -0,0 +1,41 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/Arena.h>
#include <Interpreters/Aggregator.h>
#include <Core/ColumnNumbers.h>
namespace DB
{
class ExpressionActions;
/** Takes blocks after grouping, with non-finalized aggregate functions.
* Calculates subtotals and grand totals values for a set of columns.
*/
class RollupBlockInputStream : public IProfilingBlockInputStream
{
private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
using AggregateColumns = std::vector<ColumnRawPtrs>;
public:
RollupBlockInputStream(
const BlockInputStreamPtr & input_, const Aggregator::Params & params_);
String getName() const override { return "Rollup"; }
Block getHeader() const override;
protected:
Block readImpl() override;
private:
Aggregator aggregator;
ColumnNumbers keys;
ssize_t current_key = -1;
Block rollup_block;
};
}

View File

@ -14,10 +14,10 @@ namespace DB
TotalsHavingBlockInputStream::TotalsHavingBlockInputStream(
const BlockInputStreamPtr & input_,
bool overflow_row_, const ExpressionActionsPtr & expression_,
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_)
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_)
: overflow_row(overflow_row_),
expression(expression_), filter_column_name(filter_column_), totals_mode(totals_mode_),
auto_include_threshold(auto_include_threshold_)
auto_include_threshold(auto_include_threshold_), final(final_)
{
children.push_back(input_);
@ -100,6 +100,7 @@ Block TotalsHavingBlockInputStream::getTotals()
Block TotalsHavingBlockInputStream::getHeader() const
{
Block res = children.at(0)->getHeader();
if (final)
finalize(res);
if (expression)
expression->execute(res);
@ -127,6 +128,7 @@ Block TotalsHavingBlockInputStream::readImpl()
return finalized;
finalized = block;
if (final)
finalize(finalized);
total_keys += finalized.rows();

View File

@ -26,7 +26,7 @@ public:
TotalsHavingBlockInputStream(
const BlockInputStreamPtr & input_,
bool overflow_row_, const ExpressionActionsPtr & expression_,
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_);
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_);
String getName() const override { return "TotalsHaving"; }
@ -43,6 +43,7 @@ private:
String filter_column_name;
TotalsMode totals_mode;
double auto_include_threshold;
bool final;
size_t passed_keys = 0;
size_t total_keys = 0;

View File

@ -4,6 +4,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeUUID.h>
#include <Common/typeid_cast.h>
@ -55,6 +56,8 @@ void ExternalResultDescription::init(const Block & sample_block_)
types.push_back(ValueType::Date);
else if (typeid_cast<const DataTypeDateTime *>(type))
types.push_back(ValueType::DateTime);
else if (typeid_cast<const DataTypeUUID *>(type))
types.push_back(ValueType::UUID);
else
throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE};

View File

@ -24,7 +24,8 @@ struct ExternalResultDescription
Float64,
String,
Date,
DateTime
DateTime,
UUID
};
Block sample_block;

View File

@ -276,13 +276,13 @@ void FlatDictionary::createAttributes()
void FlatDictionary::blockToAttributes(const Block & block)
{
const auto & id_column = *block.safeGetByPosition(0).column;
const IColumn & id_column = *block.safeGetByPosition(0).column;
element_count += id_column.size();
for (const auto attribute_idx : ext::range(0, attributes.size()))
for (const size_t attribute_idx : ext::range(0, attributes.size()))
{
const auto &attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
auto &attribute = attributes[attribute_idx];
const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
Attribute & attribute = attributes[attribute_idx];
for (const auto row_idx : ext::range(0, id_column.size()))
setAttributeValue(attribute, id_column[row_idx].get<UInt64>(), attribute_column[row_idx]);

View File

@ -271,9 +271,9 @@ void HashedDictionary::blockToAttributes(const Block &block)
const auto & id_column = *block.safeGetByPosition(0).column;
element_count += id_column.size();
for (const auto attribute_idx : ext::range(0, attributes.size()))
for (const size_t attribute_idx : ext::range(0, attributes.size()))
{
const auto &attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
auto & attribute = attributes[attribute_idx];
for (const auto row_idx : ext::range(0, id_column.size()))

View File

@ -135,6 +135,18 @@ namespace
static_cast<const Poco::MongoDB::ConcreteElement<Poco::Timestamp> &>(value).value().epochTime());
break;
}
case ValueType::UUID:
{
if (value.type() == Poco::MongoDB::ElementTraits<String>::TypeId)
{
String string = static_cast<const Poco::MongoDB::ConcreteElement<String> &>(value).value();
static_cast<ColumnUInt128 &>(column).getData().push_back(parse<UUID>(string));
}
else
throw Exception{"Type mismatch, expected String (UUID), got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH};
break;
}
}
}

View File

@ -4,6 +4,7 @@
#include <Dictionaries/MySQLBlockInputStream.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <ext/range.h>
#include <vector>
@ -53,6 +54,7 @@ namespace
case ValueType::String: static_cast<ColumnString &>(column).insertData(value.data(), value.size()); break;
case ValueType::Date: static_cast<ColumnUInt16 &>(column).insert(UInt16{value.getDate().getDayNum()}); break;
case ValueType::DateTime: static_cast<ColumnUInt32 &>(column).insert(time_t{value.getDateTime()}); break;
case ValueType::UUID: static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.data(), value.size())); break;
}
}

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
@ -59,6 +60,8 @@ namespace
case ValueType::String: static_cast<ColumnString &>(column).insert(value.convert<String>()); break;
case ValueType::Date: static_cast<ColumnUInt16 &>(column).insert(UInt16{LocalDate{value.convert<String>()}.getDayNum()}); break;
case ValueType::DateTime: static_cast<ColumnUInt32 &>(column).insert(time_t{LocalDateTime{value.convert<String>()}}); break;
case ValueType::UUID: static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.convert<std::string>())); break;
}
}

View File

@ -5,10 +5,12 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/Native.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
@ -16,6 +18,7 @@
#include <Core/AccurateComparison.h>
#include <Common/FieldVisitors.h>
#include <Common/typeid_cast.h>
#include <Common/Arena.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/ExpressionActions.h>
#include <ext/range.h>
@ -1145,6 +1148,14 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
/// Special case when multiply aggregate function state
if (isAggregateMultiply(arguments[0], arguments[1]))
{
if (checkDataType<DataTypeAggregateFunction>(arguments[0].get()))
return arguments[0];
return arguments[1];
}
/// 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]))
{
@ -1195,8 +1206,72 @@ public:
return type_res;
}
bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const
{
if constexpr (!std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>)
return false;
auto is_uint_type = [](const DataTypePtr & type)
{
return checkDataType<DataTypeUInt8>(type.get()) || checkDataType<DataTypeUInt16>(type.get())
|| checkDataType<DataTypeUInt32>(type.get()) || checkDataType<DataTypeUInt64>(type.get());
};
return ((checkDataType<DataTypeAggregateFunction>(type0.get()) && is_uint_type(type1))
|| (is_uint_type(type0) && checkDataType<DataTypeAggregateFunction>(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<DataTypeAggregateFunction>(block.getByPosition(new_arguments[1]).type.get()))
std::swap(new_arguments[0], new_arguments[1]);
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(new_arguments[0]).column.get());
IAggregateFunction * function = column->getAggregateFunction().get();
auto arena = std::make_shared<Arena>();
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);
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))
{

View File

@ -27,6 +27,7 @@ set (INTERNAL_COMPILER_CUSTOM_ROOT ON CACHE INTERNAL "")
set (INTERNAL_COMPILER_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_CXX_FLAGS_${CMAKE_BUILD_TYPE_UC}} ${CXX_FLAGS_INTERNAL_COMPILER} -x c++ -march=native -fPIC -fvisibility=hidden -fno-implement-inlines -nostdinc -nostdinc++ -Wno-unused-command-line-argument -Bprefix=${PATH_SHARE}/clickhouse -isysroot=${INTERNAL_COMPILER_HEADERS_ROOT}" CACHE STRING "")
# TODO: use libs from package: -nodefaultlibs -lm -lc -lgcc_s -lgcc -lc++ -lc++abi
string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS})
string(REPLACE "-no-pie" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS})
if (INTERNAL_COMPILER_NO_WARNING)
string (REPLACE "-Wall" "" INTERNAL_COMPILER_FLAGS ${INTERNAL_COMPILER_FLAGS})
@ -46,7 +47,7 @@ string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Boost_INCLUDE_DIRS ${Boost_
string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Poco_Foundation_INCLUDE_DIR ${Poco_Foundation_INCLUDE_DIR})
string (REPLACE ${ClickHouse_SOURCE_DIR} "" INTERNAL_Poco_Util_INCLUDE_DIR ${Poco_Util_INCLUDE_DIR})
message (STATUS "Using internal=${USE_INTERNAL_LLVM_LIBRARY} compiler=${USE_EMBEDDED_COMPILER}: headers=${INTERNAL_COMPILER_HEADERS} : ${INTERNAL_COMPILER_ENV} ${INTERNAL_COMPILER_EXECUTABLE} ${INTERNAL_COMPILER_FLAGS}; ${INTERNAL_LINKER_EXECUTABLE}")
message (STATUS "Using internal=${USE_INTERNAL_LLVM_LIBRARY} compiler=${USE_EMBEDDED_COMPILER}: headers=${INTERNAL_COMPILER_HEADERS} : ${INTERNAL_COMPILER_ENV} ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE} ${INTERNAL_COMPILER_FLAGS}; ${INTERNAL_LINKER_EXECUTABLE}")
set (CONFIG_COMPILE ${ClickHouse_BINARY_DIR}/dbms/src/Interpreters/config_compile.h)
configure_file (${ClickHouse_SOURCE_DIR}/dbms/src/Interpreters/config_compile.h.in ${CONFIG_COMPILE})

View File

@ -244,23 +244,29 @@ void Compiler::compile(
/// echo | g++ -x c++ -E -Wp,-v -
" -isystem " << compiler_headers_root << "/usr/include/c++/*"
#if defined(CMAKE_LIBRARY_ARCHITECTURE)
" -isystem " << compiler_headers_root << "/usr/include/" CMAKE_LIBRARY_ARCHITECTURE "/c++/*"
#endif
" -isystem " << compiler_headers_root << "/usr/include/c++/*/backward"
" -isystem " << compiler_headers_root << "/usr/include/clang/*/include" /// if compiler is clang (from package)
" -isystem " << compiler_headers_root << "/usr/local/lib/clang/*/include" /// if clang installed manually
" -isystem " << compiler_headers_root << "/usr/lib/clang/*/include" /// if clang build from submodules
#if defined(CMAKE_LIBRARY_ARCHITECTURE)
" -isystem " << compiler_headers_root << "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include-fixed"
" -isystem " << compiler_headers_root << "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include"
#endif
" -isystem " << compiler_headers_root << "/usr/local/include" /// if something installed manually
#if defined(CMAKE_LIBRARY_ARCHITECTURE)
" -isystem " << compiler_headers_root << "/usr/include/" CMAKE_LIBRARY_ARCHITECTURE
#endif
" -isystem " << compiler_headers_root << "/usr/include"
#endif
" -I " << compiler_headers << "/dbms/src/"
" -I " << compiler_headers << "/contrib/cityhash102/include/"
" -I " << compiler_headers << "/contrib/libpcg-random/include/"
" -I " << compiler_headers << INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR
" -I " << compiler_headers << INTERNAL_Poco_Foundation_INCLUDE_DIR
" -I " << compiler_headers << INTERNAL_Boost_INCLUDE_DIRS
" -isystem " << compiler_headers << "/contrib/cityhash102/include/"
" -isystem " << compiler_headers << "/contrib/libpcg-random/include/"
" -isystem " << compiler_headers << INTERNAL_DOUBLE_CONVERSION_INCLUDE_DIR
" -isystem " << compiler_headers << INTERNAL_Poco_Foundation_INCLUDE_DIR
" -isystem " << compiler_headers << INTERNAL_Boost_INCLUDE_DIRS
" -I " << compiler_headers << "/libs/libcommon/include/"
" " << additional_compiler_flags <<
" -shared -o " << so_tmp_file_path << " " << cpp_file_path

View File

@ -133,6 +133,14 @@ ExpressionAction ExpressionAction::project(const Names & projected_columns_)
return a;
}
ExpressionAction ExpressionAction::addAliases(const NamesWithAliases & aliased_columns_)
{
ExpressionAction a;
a.type = ADD_ALIASES;
a.projection = aliased_columns_;
return a;
}
ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context)
{
if (array_joined_columns.empty())
@ -256,6 +264,8 @@ void ExpressionAction::prepare(Block & sample_block)
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
ColumnWithTypeAndName column = sample_block.getByName(name);
if (column.column)
column.column = (*std::move(column.column)).mutate();
if (alias != "")
column.name = alias;
new_block.insert(std::move(column));
@ -265,6 +275,19 @@ void ExpressionAction::prepare(Block & sample_block)
break;
}
case ADD_ALIASES:
{
for (size_t i = 0; i < projection.size(); ++i)
{
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
const ColumnWithTypeAndName & column = sample_block.getByName(name);
if (alias != "" && !sample_block.has(alias))
sample_block.insert({column.column, column.type, alias});
}
break;
}
case REMOVE_COLUMN:
{
sample_block.erase(source_name);
@ -438,6 +461,8 @@ void ExpressionAction::execute(Block & block, std::unordered_map<std::string, si
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
ColumnWithTypeAndName column = block.getByName(name);
if (column.column)
column.column = (*std::move(column.column)).mutate();
if (alias != "")
column.name = alias;
new_block.insert(std::move(column));
@ -448,6 +473,19 @@ void ExpressionAction::execute(Block & block, std::unordered_map<std::string, si
break;
}
case ADD_ALIASES:
{
for (size_t i = 0; i < projection.size(); ++i)
{
const std::string & name = projection[i].first;
const std::string & alias = projection[i].second;
const ColumnWithTypeAndName & column = block.getByName(name);
if (alias != "" && !block.has(alias))
block.insert({column.column, column.type, alias});
}
break;
}
case REMOVE_COLUMN:
block.erase(source_name);
break;
@ -529,8 +567,9 @@ std::string ExpressionAction::toString() const
}
break;
case PROJECT:
ss << "PROJECT ";
case PROJECT: [[fallthrough]];
case ADD_ALIASES:
ss << (type == PROJECT ? "PROJECT " : "ADD_ALIASES ");
for (size_t i = 0; i < projection.size(); ++i)
{
if (i)
@ -786,6 +825,16 @@ void ExpressionActions::finalize(const Names & output_columns)
needed_columns = NameSet(in.begin(), in.end());
unmodified_columns.clear();
}
else if (action.type == ExpressionAction::ADD_ALIASES)
{
needed_columns.insert(in.begin(), in.end());
for (auto & name_wit_alias : action.projection)
{
auto it = unmodified_columns.find(name_wit_alias.second);
if (it != unmodified_columns.end())
unmodified_columns.erase(it);
}
}
else if (action.type == ExpressionAction::ARRAY_JOIN)
{
/// Do not ARRAY JOIN columns that are not used anymore.

View File

@ -61,6 +61,8 @@ public:
/// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result.
PROJECT,
/// Add columns with alias names. This columns are the same as non-aliased. PROJECT columns if you need to modify them.
ADD_ALIASES,
};
Type type;
@ -106,6 +108,7 @@ public:
static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name);
static ExpressionAction project(const NamesWithAliases & projected_columns_);
static ExpressionAction project(const Names & projected_columns_);
static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_);
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
static ExpressionAction ordinaryJoin(std::shared_ptr<const Join> join_, const Names & join_key_names_left,
const NamesAndTypesList & columns_added_by_join_);

View File

@ -2894,7 +2894,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
{
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(source_columns, settings);
NamesWithAliases result_columns;
@ -2911,7 +2911,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
{
std::string name = asts[i]->getColumnName();
std::string alias;
if (project_result)
if (add_aliases)
alias = asts[i]->getAliasOrColumnName();
else
alias = name;
@ -2920,11 +2920,15 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
getRootActions(asts[i], false, false, actions);
}
if (project_result)
if (add_aliases)
{
if (project_result)
actions->add(ExpressionAction::project(result_columns));
}
else
actions->add(ExpressionAction::addAliases(result_columns));
}
if (!(add_aliases && project_result))
{
/// We will not delete the original columns.
for (const auto & column_name_type : source_columns)

View File

@ -155,9 +155,10 @@ public:
void appendProjectResult(ExpressionActionsChain & chain) const;
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
/// If project_result, only the calculated values in the desired order, renamed to aliases, remain in the output block.
/// If add_aliases, only the calculated values in the desired order and add aliases.
/// If also project_result, than only aliases remain in the output block.
/// Otherwise, only temporary columns will be deleted from the block.
ExpressionActionsPtr getActions(bool project_result);
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true);
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
/// Does not execute subqueries.

View File

@ -594,6 +594,11 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
current_dependents[proj.first].emplace();
break;
case ExpressionAction::ADD_ALIASES:
for (const auto & proj : actions[i].projection)
current_dependents[proj.first].emplace();
break;
case ExpressionAction::ADD_COLUMN:
case ExpressionAction::COPY_COLUMN:
case ExpressionAction::ARRAY_JOIN:

View File

@ -18,6 +18,7 @@
#include <DataStreams/CreatingSetsBlockInputStream.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/ConcatBlockInputStream.h>
#include <DataStreams/RollupBlockInputStream.h>
#include <DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h>
#include <Parsers/ASTSelectQuery.h>
@ -479,7 +480,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
bool aggregate_final =
expressions.need_aggregate &&
to_stage > QueryProcessingStage::WithMergeableState &&
!query.group_by_with_totals;
!query.group_by_with_totals && !query.group_by_with_rollup;
if (expressions.first_stage)
{
@ -535,7 +536,13 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
executeMergeAggregated(pipeline, aggregate_overflow_row, aggregate_final);
if (!aggregate_final)
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row);
{
if (query.group_by_with_totals)
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, !query.group_by_with_rollup);
if (query.group_by_with_rollup)
executeRollup(pipeline);
}
else if (expressions.has_having)
executeHaving(pipeline, expressions.before_having);
@ -549,7 +556,10 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream();
if (query.group_by_with_totals && !aggregate_final)
executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row);
executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup);
if (query.group_by_with_rollup && !aggregate_final)
executeRollup(pipeline);
}
if (expressions.has_order_by)
@ -627,6 +637,9 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz
void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, const PrewhereInfoPtr & prewhere_info)
{
const Settings & settings = context.getSettingsRef();
/// Actions to calculate ALIAS if required.
ExpressionActionsPtr alias_actions;
/// Are ALIAS columns required for query execution?
@ -647,26 +660,108 @@ void InterpreterSelectQuery::executeFetchColumns(
if (alias_columns_required)
{
/// Columns required for prewhere actions.
NameSet required_prewhere_columns;
/// Columns required for prewhere actions which are aliases in storage.
NameSet required_prewhere_aliases;
Block prewhere_actions_result;
if (prewhere_info)
{
auto required_columns = prewhere_info->prewhere_actions->getRequiredColumns();
required_prewhere_columns.insert(required_columns.begin(), required_columns.end());
prewhere_actions_result = prewhere_info->prewhere_actions->getSampleBlock();
}
/// We will create an expression to return all the requested columns, with the calculation of the required ALIAS columns.
auto required_columns_expr_list = std::make_shared<ASTExpressionList>();
/// Separate expression for columns used in prewhere.
auto required_prewhere_columns_expr_list = std::make_shared<ASTExpressionList>();
/// Columns which we will get after prewhere execution.
auto source_columns = storage->getColumns().getAllPhysical();
for (const auto & column : required_columns)
{
ASTPtr column_expr;
const auto default_it = column_defaults.find(column);
if (default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias)
required_columns_expr_list->children.emplace_back(setAlias(default_it->second.expression->clone(), column));
bool is_alias = default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias;
if (is_alias)
column_expr = setAlias(default_it->second.expression->clone(), column);
else
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(column));
column_expr = std::make_shared<ASTIdentifier>(column);
if (required_prewhere_columns.count(column))
{
required_prewhere_columns_expr_list->children.emplace_back(std::move(column_expr));
if (is_alias)
required_prewhere_aliases.insert(column);
}
else
required_columns_expr_list->children.emplace_back(std::move(column_expr));
}
alias_actions = ExpressionAnalyzer(required_columns_expr_list, context, storage).getActions(true);
/// Add columns which will be added by prewhere (otherwise we will remove them in project action).
for (const auto & column : prewhere_actions_result)
{
if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name)
continue;
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
source_columns.emplace_back(column.name, column.type);
}
alias_actions = ExpressionAnalyzer(required_columns_expr_list, context, nullptr, source_columns).getActions(true);
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
required_columns = alias_actions->getRequiredColumns();
/// Do not remove prewhere filter if it is a column which is used as alias.
if (prewhere_info && prewhere_info->remove_prewhere_column)
if (required_columns.end()
!= std::find(required_columns.begin(), required_columns.end(), prewhere_info->prewhere_column_name))
prewhere_info->remove_prewhere_column = false;
/// Remove columns which will be added by prewhere.
for (size_t i = 0; i < required_columns.size(); ++i)
{
if (!storage->getColumns().hasPhysical(required_columns[i]))
{
std::swap(required_columns[i], required_columns.back());
required_columns.pop_back();
}
}
if (prewhere_info)
{
/// Don't remove columns which are needed to be aliased.
auto new_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), settings);
for (const auto & action : prewhere_info->prewhere_actions->getActions())
{
if (action.type != ExpressionAction::REMOVE_COLUMN
|| required_columns.end() == std::find(required_columns.begin(), required_columns.end(), action.source_name))
new_actions->add(action);
}
prewhere_info->prewhere_actions = std::move(new_actions);
prewhere_info->alias_actions = ExpressionAnalyzer(required_prewhere_columns_expr_list, context, storage).getActions(true, false);
/// Add columns required by alias actions.
auto required_aliased_columns = prewhere_info->alias_actions->getRequiredColumns();
for (auto & column : required_aliased_columns)
if (!prewhere_actions_result.has(column))
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column))
required_columns.push_back(column);
/// Add columns required by prewhere actions.
for (const auto & column : required_prewhere_columns)
if (required_prewhere_aliases.count(column) == 0)
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column))
required_columns.push_back(column);
}
}
}
const Settings & settings = context.getSettingsRef();
/// Limitation on the number of columns to read.
/// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns.
@ -959,7 +1054,7 @@ void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const Expression
}
void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row)
void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{
executeUnion(pipeline);
@ -967,7 +1062,34 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>(
pipeline.firstStream(), overflow_row, expression,
has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold);
has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final);
}
void InterpreterSelectQuery::executeRollup(Pipeline & pipeline)
{
executeUnion(pipeline);
Names key_names;
AggregateDescriptions aggregates;
query_analyzer->getAggregateInfo(key_names, aggregates);
Block header = pipeline.firstStream()->getHeader();
ColumnNumbers keys;
for (const auto & name : key_names)
keys.push_back(header.getPositionByName(name));
const Settings & settings = context.getSettingsRef();
Aggregator::Params params(header, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile,
SettingUInt64(0), SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath());
pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params);
}

View File

@ -177,7 +177,7 @@ private:
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final);
void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row);
void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(Pipeline & pipeline);
@ -190,6 +190,7 @@ private:
void executeDistinct(Pipeline & pipeline, bool before_order, Names columns);
void executeExtremes(Pipeline & pipeline);
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
void executeRollup(Pipeline & pipeline);
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
*

View File

@ -19,7 +19,6 @@
#include <Common/CurrentThread.h>
#include <Interpreters/QueryPriorities.h>
#include <Interpreters/ClientInfo.h>
#include <Common/ThreadStatus.h>
#include <DataStreams/BlockIO.h>

View File

@ -203,6 +203,7 @@ void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group)
void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group)
{
get()->attachQuery(thread_group, false);
getScope()->deleter = CurrentThread::defaultThreadDeleter;
}
std::string CurrentThread::getCurrentQueryID()

View File

@ -1,9 +1,6 @@
#pragma once
#cmakedefine CMAKE_LIBRARY_ARCHITECTURE "@CMAKE_LIBRARY_ARCHITECTURE@"
#if !defined(CMAKE_LIBRARY_ARCHITECTURE)
#define CMAKE_LIBRARY_ARCHITECTURE ""
#endif
#cmakedefine PATH_SHARE "@PATH_SHARE@"
#cmakedefine INTERNAL_COMPILER_FLAGS "@INTERNAL_COMPILER_FLAGS@"
#cmakedefine INTERNAL_COMPILER_BIN_ROOT "@INTERNAL_COMPILER_BIN_ROOT@"

View File

@ -103,6 +103,9 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
: typeid_cast<const ASTExpressionList &>(*group_expression_list).formatImplMultiline(s, state, frame);
}
if (group_by_with_rollup)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH ROLLUP" << (s.hilite ? hilite_none : "");
if (group_by_with_totals)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : "");

View File

@ -28,6 +28,7 @@ public:
ASTPtr where_expression;
ASTPtr group_expression_list;
bool group_by_with_totals = false;
bool group_by_with_rollup = false;
ASTPtr having_expression;
ASTPtr order_expression_list;
ASTPtr limit_by_value;

View File

@ -39,6 +39,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_limit("LIMIT");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_by("BY");
ParserKeyword s_rollup("ROLLUP");
ParserKeyword s_top("TOP");
ParserKeyword s_offset("OFFSET");
@ -48,6 +49,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserExpressionWithOptionalAlias exp_elem(false);
ParserOrderByExpressionList order_list;
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
/// WITH expr list
{
if (s_with.ignore(pos, expected))
@ -67,8 +71,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (s_top.ignore(pos, expected))
{
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
ParserNumber num;
if (open_bracket.ignore(pos, expected))
@ -113,14 +115,35 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
/// GROUP BY expr list
if (s_group_by.ignore(pos, expected))
{
if (s_rollup.ignore(pos, expected))
{
select_query->group_by_with_rollup = true;
if (!open_bracket.ignore(pos, expected))
return false;
}
if (!exp_list.parse(pos, select_query->group_expression_list, expected))
return false;
if (select_query->group_by_with_rollup && !close_bracket.ignore(pos, expected))
return false;
}
/// WITH ROLLUP
if (s_with.ignore(pos, expected))
{
if (s_rollup.ignore(pos, expected))
select_query->group_by_with_rollup = true;
else if (s_totals.ignore(pos, expected))
select_query->group_by_with_totals = true;
else
return false;
}
/// WITH TOTALS
if (s_with.ignore(pos, expected))
{
if (!s_totals.ignore(pos, expected))
if (select_query->group_by_with_totals || !s_totals.ignore(pos, expected))
return false;
select_query->group_by_with_totals = true;

View File

@ -425,6 +425,7 @@ void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c)
void StorageKafka::streamThread()
{
setThreadName("KafkaStreamThr");
CurrentThread::initializeQuery();
while (!stream_cancelled)
{

View File

@ -121,26 +121,28 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
if (reader->getColumns().empty())
{
task->range_reader = MergeTreeRangeReader(
pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions,
pre_reader.get(), index_granularity, nullptr,
prewhere_info->alias_actions, prewhere_info->prewhere_actions,
&prewhere_info->prewhere_column_name, &task->ordered_names,
task->should_reorder, task->remove_prewhere_column, true);
}
else
{
task->pre_range_reader = MergeTreeRangeReader(
pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions,
pre_reader.get(), index_granularity, nullptr,
prewhere_info->alias_actions, prewhere_info->prewhere_actions,
&prewhere_info->prewhere_column_name, &task->ordered_names,
task->should_reorder, task->remove_prewhere_column, false);
task->range_reader = MergeTreeRangeReader(
reader.get(), index_granularity, &task->pre_range_reader, nullptr,
reader.get(), index_granularity, &task->pre_range_reader, nullptr, nullptr,
nullptr, &task->ordered_names, true, false, true);
}
}
else
{
task->range_reader = MergeTreeRangeReader(
reader.get(), index_granularity, nullptr, nullptr,
reader.get(), index_granularity, nullptr, nullptr, nullptr,
nullptr, &task->ordered_names, task->should_reorder, false, true);
}
}
@ -218,6 +220,9 @@ void MergeTreeBaseBlockInputStream::executePrewhereActions(Block & block, const
{
if (prewhere_info)
{
if (prewhere_info->alias_actions)
prewhere_info->alias_actions->execute(block);
prewhere_info->prewhere_actions->execute(block);
if (prewhere_info->remove_prewhere_column)
block.erase(prewhere_info->prewhere_column_name);

View File

@ -109,6 +109,9 @@ try
if (prewhere_info)
{
if (prewhere_info->alias_actions)
pre_column_names = prewhere_info->alias_actions->getRequiredColumns();
else
pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns();
if (pre_column_names.empty())

View File

@ -735,6 +735,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
rows_sources_write_buf->next();
rows_sources_uncompressed_write_buf->next();
size_t rows_sources_count = rows_sources_write_buf->count();
/// In special case, when there is only one source part, and no rows were skipped, we may have
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
/// number of input rows.
if ((rows_sources_count > 0 || parts.size() > 1) && sum_input_rows_exact != rows_sources_count)
throw Exception("Number of rows in source parts (" + toString(sum_input_rows_exact)
+ ") differs from number of bytes written to rows_sources file (" + toString(rows_sources_count)
+ "). It is a bug.", ErrorCodes::LOGICAL_ERROR);
CompressedReadBufferFromFile rows_sources_read_buf(rows_sources_file_path, 0, 0);
for (size_t column_num = 0, gathering_column_names_size = gathering_column_names.size();

View File

@ -366,13 +366,13 @@ void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter)
MergeTreeRangeReader::MergeTreeRangeReader(
MergeTreeReader * merge_tree_reader, size_t index_granularity,
MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions,
MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader,
ExpressionActionsPtr alias_actions, ExpressionActionsPtr prewhere_actions,
const String * prewhere_column_name, const Names * ordered_names,
bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain)
: index_granularity(index_granularity), merge_tree_reader(merge_tree_reader)
, prev_reader(prev_reader), prewhere_column_name(prewhere_column_name)
, ordered_names(ordered_names), prewhere_actions(std::move(prewhere_actions))
, ordered_names(ordered_names), alias_actions(alias_actions), prewhere_actions(std::move(prewhere_actions))
, always_reorder(always_reorder), remove_prewhere_column(remove_prewhere_column)
, last_reader_in_chain(last_reader_in_chain), is_initialized(true)
{
@ -571,6 +571,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
if (!prewhere_actions)
return;
if (alias_actions)
alias_actions->execute(result.block);
prewhere_actions->execute(result.block);
auto & prewhere_column = result.block.getByName(*prewhere_column_name);
size_t prev_rows = result.block.rows();

View File

@ -19,8 +19,8 @@ class MergeTreeReader;
class MergeTreeRangeReader
{
public:
MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity,
MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions,
MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader,
ExpressionActionsPtr alias_actions, ExpressionActionsPtr prewhere_actions,
const String * prewhere_column_name, const Names * ordered_names,
bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain);
@ -175,6 +175,7 @@ private:
const String * prewhere_column_name = nullptr;
const Names * ordered_names = nullptr;
ExpressionActionsPtr alias_actions = nullptr; /// If not nullptr, calculate aliases.
ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter.
Stream stream;

View File

@ -196,6 +196,9 @@ std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(
if (prewhere_info)
{
/// collect columns required for PREWHERE evaluation
if (prewhere_info->alias_actions)
required_pre_column_names = prewhere_info->alias_actions->getRequiredColumns();
else
required_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns();
/// there must be at least one column required for PREWHERE

View File

@ -21,6 +21,8 @@ using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointers
struct PrewhereInfo
{
/// Ections which are executed in order to alias columns are used for prewhere actions.
ExpressionActionsPtr alias_actions;
/// Actions which are executed on block in order to get filter column for prewhere step.
ExpressionActionsPtr prewhere_actions;
String prewhere_column_name;

View File

@ -3431,8 +3431,6 @@ void StorageReplicatedMergeTree::drop()
if (is_readonly || !zookeeper)
throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
// checkTableCanBeDropped(); // uncomment to feel yourself safe
shutdown();
if (zookeeper->expired())

View File

@ -27,5 +27,5 @@ if (ENABLE_TESTS)
# maybe add --no-long ?
# if you want disable some tests: env TEST_OPT0='--skip compile'
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} INTERNAL_COMPILER_BIN_ROOT=${INTERNAL_COMPILER_BIN_ROOT} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endif ()

View File

@ -75,7 +75,7 @@ fi
VERSION=`${BIN_DIR}clickhouse-client --version-clean`
# If run from compile dir - use in-place compile binary and headers
[ -n "$BIN_DIR" ] && INTERNAL_COMPILER_PARAMS="--compiler_executable_root=$BUILD_DIR/dbms/programs/ --compiler_headers=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/ --compiler_headers_root=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/"
[ -n "$BIN_DIR" ] && INTERNAL_COMPILER_PARAMS="--compiler_executable_root=${INTERNAL_COMPILER_BIN_ROOT:=$BUILD_DIR/dbms/programs/} --compiler_headers=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/ --compiler_headers_root=$BUILD_DIR/dbms/programs/clang/headers/$VERSION/"
$GDB ${BIN_DIR}clickhouse-server --config-file=$CLICKHOUSE_CONFIG --log=$CLICKHOUSE_LOG $TEST_SERVER_PARAMS -- \
--http_port=$CLICKHOUSE_PORT_HTTP \

View File

@ -145,7 +145,7 @@ types = [
'Int8', 'Int16', 'Int32', 'Int64',
'Float32', 'Float64',
'String',
'Date', 'DateTime'
'Date', 'DateTime', 'UUID'
]
@ -154,7 +154,7 @@ explicit_defaults = [
'-42', '-42', '-42', '-42',
'1.5', '1.6',
"'explicit-default'",
"'2015-01-01'", "'2015-01-01 00:00:00'"
"'2015-01-01'", "'2015-01-01 00:00:00'", "'550e8400-e29b-41d4-a716-446655440000'"
]
@ -163,7 +163,7 @@ implicit_defaults = [
'-1', '-1', '-1', '-1',
'2.71828', '2.71828',
'implicit-default',
'2015-11-25', '2015-11-25 00:00:00'
'2015-11-25', '2015-11-25 00:00:00', "550e8400-e29b-41d4-a716-446655440000"
]
@ -210,7 +210,7 @@ def generate_data(args):
'Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64,'
'Float32_ Float32, Float64_ Float64,'
'String_ String,'
'Date_ Date, DateTime_ DateTime, Parent UInt64'
'Date_ Date, DateTime_ DateTime, Parent UInt64, UUID_ UUID'
') engine=Log; insert into test.dictionary_source format TabSeparated'
'"'.format(source = args.source, ch = args.client, port = args.port))
@ -232,7 +232,7 @@ def generate_data(args):
'UInt8_ tinyint unsigned, UInt16_ smallint unsigned, UInt32_ int unsigned, UInt64_ bigint unsigned, '
'Int8_ tinyint, Int16_ smallint, Int32_ int, Int64_ bigint, '
'Float32_ float, Float64_ double, '
'String_ text, Date_ date, DateTime_ datetime, Parent bigint unsigned'
'String_ text, Date_ date, DateTime_ datetime, Parent bigint unsigned, UUID_ varchar(36)'
');'
'load data local infile \'{0}/source.tsv\' into table test.dictionary_source;" | mysql $MYSQL_OPTIONS --local-infile=1'
.format(prefix), shell=True)

View File

@ -0,0 +1,8 @@
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440001
550e8400-e29b-41d4-a716-446655440002
550e8400-e29b-41d4-a716-446655440003
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000

View File

@ -0,0 +1,8 @@
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440001
550e8400-e29b-41d4-a716-446655440002
550e8400-e29b-41d4-a716-446655440003
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000
550e8400-e29b-41d4-a716-446655440000

View File

@ -1,4 +1,4 @@
0 0 0 0 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0
1 1 1 1 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0
2 2 2 2 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1
3 3 3 3 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2
0 0 0 0 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0 550e8400-e29b-41d4-a716-446655440000
1 1 1 1 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0 550e8400-e29b-41d4-a716-446655440001
2 2 2 2 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1 550e8400-e29b-41d4-a716-446655440002
3 3 3 3 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2 550e8400-e29b-41d4-a716-446655440003

1 0 0 0 0 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0 550e8400-e29b-41d4-a716-446655440000
2 1 1 1 1 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0 550e8400-e29b-41d4-a716-446655440001
3 2 2 2 2 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1 550e8400-e29b-41d4-a716-446655440002
4 3 3 3 3 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2 550e8400-e29b-41d4-a716-446655440003

View File

@ -0,0 +1,48 @@
#!/usr/bin/env bash
set -e
# Get all server logs
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="trace"
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
cur_name=${BASH_SOURCE[0]}
server_logs_file=$cur_name"_server.logs"
server_logs="--server_logs_file=$server_logs_file"
rm -f "$server_logs_file"
settings="$server_logs --log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1"
# Check that logs from remote servers are passed from client
# SELECT
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM system.one FORMAT Null"
lines_one_server=`cat "$server_logs_file" | wc -l`
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM remote('127.0.0.2,127.0.0.3', system, one) FORMAT Null"
lines_two_servers=`cat "$server_logs_file" | wc -l`
(( $lines_two_servers >= 2 * $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server"
# INSERT
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null"
$CLICKHOUSE_CLIENT $settings -q "CREATE TABLE test.null (i Int8) ENGINE = Null"
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "INSERT INTO test.null VALUES (0)"
lines_one_server=`cat "$server_logs_file" | wc -l`
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'null') VALUES (0)"
lines_two_servers=`cat "$server_logs_file" | wc -l`
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null"
(( $lines_two_servers > $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server"
# Clean
rm "$server_logs_file"

View File

@ -101,35 +101,5 @@ WHERE
NOT (PVq <= PVt AND PVt <= 1.1 * PVq)
"
# Check that logs from remote servers are passed from client
# SELECT
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM system.one FORMAT Null"
lines_one_server=`cat "$server_logs_file" | wc -l`
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM remote('127.0.0.2,127.0.0.3', system, one) FORMAT Null"
lines_two_servers=`cat "$server_logs_file" | wc -l`
(( $lines_two_servers >= 2 * $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server"
# INSERT
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null"
$CLICKHOUSE_CLIENT $settings -q "CREATE TABLE test.null (i Int8) ENGINE = Null"
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "INSERT INTO test.null VALUES (0)"
lines_one_server=`cat "$server_logs_file" | wc -l`
> "$server_logs_file"
$CLICKHOUSE_CLIENT $settings -q "INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'null') VALUES (0)"
lines_two_servers=`cat "$server_logs_file" | wc -l`
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null"
(( $lines_two_servers > $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server"
# Clean
rm "$server_logs_file"

View File

@ -0,0 +1,13 @@
2
0
33
2
0
18
20 4
2
3
2
[1,1,1,1,1]
[1,1]
[1]

View File

@ -0,0 +1,22 @@
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));
SELECT countMerge(x) AS y FROM ( SELECT 2 * countState() AS x FROM ( SELECT 1 ));
SELECT countMerge(x) AS y FROM ( SELECT 0 * countState() AS x FROM ( SELECT 1 UNION ALL SELECT 2));
SELECT sumMerge(y) AS z FROM ( SELECT 3 * sumState(x) * 2 AS y FROM ( SELECT 1 AS x UNION ALL SELECT 2 AS x));
DROP TABLE IF EXISTS test.mult_aggregation;
CREATE TABLE test.mult_aggregation(a UInt32, b UInt32) ENGINE = Memory;
INSERT INTO test.mult_aggregation VALUES(1, 1);
INSERT INTO test.mult_aggregation VALUES(1, 3);
SELECT sumMerge(x * 5), sumMerge(x) FROM (SELECT sumState(b) AS x FROM test.mult_aggregation);
SELECT uniqMerge(x * 10) FROM (SELECT uniqState(b) AS x FROM test.mult_aggregation);
SELECT maxMerge(x * 10) FROM (SELECT maxState(b) AS x FROM test.mult_aggregation);
SELECT avgMerge(x * 10) FROM (SELECT avgState(b) AS x FROM test.mult_aggregation);
SELECT groupArrayMerge(y * 5) FROM (SELECT groupArrayState(x) AS y FROM (SELECT 1 AS x));
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));
DROP TABLE IF EXISTS test.mult_aggregation;

View File

@ -0,0 +1,27 @@
0 120 8
a 0 70 4
a 1 25 2
a 2 45 2
b 0 50 4
b 1 15 2
b 2 35 2
0 120 8
a 0 70 4
a 1 25 2
a 2 45 2
b 0 50 4
b 1 15 2
b 2 35 2
0 120 8
120 8
a 70 4
b 50 4
120 8
a 70 4
b 50 4
120 8
a 70 4
b 50 4
120 8

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS test.rollup;
CREATE TABLE test.rollup(a String, b Int32, s Int32) ENGINE = Memory;
INSERT INTO test.rollup VALUES('a', 1, 10);
INSERT INTO test.rollup VALUES('a', 1, 15);
INSERT INTO test.rollup VALUES('a', 2, 20);
INSERT INTO test.rollup VALUES('a', 2, 25);
INSERT INTO test.rollup VALUES('b', 1, 10);
INSERT INTO test.rollup VALUES('b', 1, 5);
INSERT INTO test.rollup VALUES('b', 2, 20);
INSERT INTO test.rollup VALUES('b', 2, 15);
SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) ORDER BY a, b;
SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) WITH TOTALS ORDER BY a, b;
SELECT a, sum(s), count() from test.rollup GROUP BY ROLLUP(a) ORDER BY a;
SELECT a, sum(s), count() from test.rollup GROUP BY a WITH ROLLUP ORDER BY a;
SELECT a, sum(s), count() from test.rollup GROUP BY a WITH ROLLUP WITH TOTALS ORDER BY a;

View File

@ -0,0 +1,30 @@
-
1
1
1
2
-
1
1
1
2
-
1
1
1
2
-
1
1
1
2
-
1
1
1
2
-
1
1
1
2

View File

@ -0,0 +1,35 @@
drop table if exists test.prewhere_alias;
CREATE TABLE test.prewhere_alias (a UInt8, b Int32, c UInt8 ALIAS a, d Int64 ALIAS b + 1, e Int32 alias a + b) ENGINE = MergeTree ORDER BY tuple();
insert into test.prewhere_alias values (1, 1);
select '-';
select a from test.prewhere_alias prewhere a = 1;
select b from test.prewhere_alias prewhere a = 1;
select c from test.prewhere_alias prewhere a = 1;
select d from test.prewhere_alias prewhere a = 1;
select '-';
select a from test.prewhere_alias prewhere b = 1;
select b from test.prewhere_alias prewhere b = 1;
select c from test.prewhere_alias prewhere b = 1;
select d from test.prewhere_alias prewhere b = 1;
select '-';
select a from test.prewhere_alias prewhere c = 1;
select b from test.prewhere_alias prewhere c = 1;
select c from test.prewhere_alias prewhere c = 1;
select d from test.prewhere_alias prewhere c = 1;
select '-';
select a from test.prewhere_alias prewhere d = 2;
select b from test.prewhere_alias prewhere d = 2;
select c from test.prewhere_alias prewhere d = 2;
select d from test.prewhere_alias prewhere d = 2;
select '-';
select a from test.prewhere_alias prewhere a;
select b from test.prewhere_alias prewhere a;
select c from test.prewhere_alias prewhere a;
select d from test.prewhere_alias prewhere a;
select '-';
select a from test.prewhere_alias prewhere c;
select b from test.prewhere_alias prewhere c;
select c from test.prewhere_alias prewhere c;
select d from test.prewhere_alias prewhere c;
drop table if exists test.prewhere_alias;

View File

@ -0,0 +1,3 @@
k1 k1v2 1
k3 k3v1 1
k4 k4v1 -1

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS test.collapsing;
CREATE TABLE test.collapsing(key String, value String, sign Int8) ENGINE CollapsingMergeTree(sign)
ORDER BY key
SETTINGS enable_vertical_merge_algorithm=1,
vertical_merge_algorithm_min_rows_to_activate=0,
vertical_merge_algorithm_min_columns_to_activate=0;
INSERT INTO test.collapsing VALUES ('k1', 'k1v1', 1);
INSERT INTO test.collapsing VALUES ('k1', 'k1v1', -1), ('k1', 'k1v2', 1);
INSERT INTO test.collapsing VALUES ('k2', 'k2v1', 1), ('k2', 'k2v1', -1), ('k3', 'k3v1', 1);
INSERT INTO test.collapsing VALUES ('k4', 'k4v1', -1), ('k4', 'k4v2', 1), ('k4', 'k4v2', -1);
OPTIMIZE TABLE test.collapsing PARTITION tuple() FINAL;
SELECT * FROM test.collapsing ORDER BY key;
DROP TABLE test.collapsing;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (18.12.2) unstable; urgency=low
clickhouse (18.12.5) unstable; urgency=low
* Modified source code
-- <root@yandex-team.ru> Wed, 05 Sep 2018 00:28:49 +0300
-- <root@yandex-team.ru> Thu, 06 Sep 2018 07:25:55 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.12.2
ARG version=18.12.5
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.12.2
ARG version=18.12.5
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.12.2
ARG version=18.12.5
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -70,6 +70,7 @@ def build_for_lang(lang, args):
cfg = config.load_config(
config_file=config_path,
site_name='ClickHouse Documentation' if lang == 'en' else 'Документация ClickHouse',
site_url='https://clickhouse.yandex/docs/en/' if lang == 'en' else 'https://clickhouse.yandex/docs/ru/',
docs_dir=os.path.join(args.docs_dir, lang),
site_dir=os.path.join(args.output_dir, lang),
strict=True,

View File

@ -11,6 +11,8 @@
# env DIST=unstable EXTRAPACKAGES="clang-7 libstdc++-8-dev lld-7 liblld-7-dev libclang-7-dev liblld-7" DEB_CC=clang-7 DEB_CXX=clang++-7 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release
# Clang6 without internal compiler (for low memory arm64):
# env DIST=bionic DISABLE_PARALLEL=1 EXTRAPACKAGES="clang-6.0 libstdc++-8-dev" DEB_CC=clang-6.0 DEB_CXX=clang++-6.0 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release
# Do not compile internal compiler but use from system:
# env CMAKE_FLAGS="-DUSE_INTERNAL_LLVM_LIBRARY=0 -DENABLE_EMBEDDED_COMPILER=0 -DINTERNAL_COMPILER_EXECUTABLE=clang-6.0 -DINTERNAL_LINKER_EXECUTABLE=ld.lld-6.0 -DINTERNAL_COMPILER_BIN_ROOT=/usr/bin/" EXTRAPACKAGES="clang-6.0 lld-6.0 libstdc++-8-dev" DEB_CXX=clang++-6.0 DEB_CC=clang-6.0 TEST_RUN=1 TEST_OPT="compile" ./release
# Build with ASan:
# env SANITIZER=address ./release

View File

@ -3,13 +3,10 @@
<sitemap>
<loc>https://clickhouse.yandex/docs/ru/sitemap.xml</loc>
</sitemap>
<sitemap>
<loc>https://clickhouse.yandex/docs/ru/single_page/sitemap.xml</loc>
</sitemap>
<sitemap>
<loc>https://clickhouse.yandex/docs/en/sitemap.xml</loc>
</sitemap>
<sitemap>
<loc>https://clickhouse.yandex/docs/en/single_pabe/sitemap.xml</loc>
<loc>https://clickhouse.yandex/docs/sitemap_static.xml</loc>
</sitemap>
</sitemapindex>

View File

@ -0,0 +1,23 @@
<?xml version="1.0" encoding="UTF-8"?>
<urlset xmlns="http://www.sitemaps.org/schemas/sitemap/0.9">
<url>
<loc>https://clickhouse.yandex/</loc>
<changefreq>daily</changefreq>
</url>
<url>
<loc>https://clickhouse.yandex/benchmark.html</loc>
<changefreq>daily</changefreq>
</url>
<url>
<loc>https://clickhouse.yandex/tutorial.html</loc>
<changefreq>daily</changefreq>
</url>
<url>
<loc>https://clickhouse.yandex/blog/en</loc>
<changefreq>daily</changefreq>
</url>
<url>
<loc>https://clickhouse.yandex/blog/ru</loc>
<changefreq>daily</changefreq>
</url>
</urlset>