Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
BayoNet 2019-01-14 09:06:40 +03:00
commit 3222dd92f7
26 changed files with 170 additions and 164 deletions

View File

@ -90,8 +90,6 @@ if (GLIBC_COMPATIBILITY)
set (USE_INTERNAL_MEMCPY ON)
endif ()
set (COMPILER_FLAGS "${COMPILER_FLAGS}")
string(REGEX MATCH "-?[0-9]+(.[0-9]+)?$" COMPILER_POSTFIX ${CMAKE_CXX_COMPILER})
find_program (LLD_PATH NAMES "lld${COMPILER_POSTFIX}" "lld")
@ -108,10 +106,15 @@ if (LINKER_NAME)
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}")
endif ()
option (PIPE "-pipe compiler option [less /tmp usage, more ram usage]" ON)
if (PIPE)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -pipe")
endif ()
cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd
if(NOT AVAILABLE_PHYSICAL_MEMORY OR AVAILABLE_PHYSICAL_MEMORY GREATER 8000)
option(COMPILER_PIPE "-pipe compiler option [less /tmp usage, more ram usage]" ON)
endif()
if(COMPILER_PIPE)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -pipe")
else()
message(STATUS "Disabling compiler -pipe option (have only ${AVAILABLE_PHYSICAL_MEMORY} mb of memory)")
endif()
include (cmake/test_cpu.cmake)

View File

@ -6,7 +6,7 @@ set (OPENSSL_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES})
if (NOT USE_INTERNAL_SSL_LIBRARY)
if (APPLE)
set (OPENSSL_ROOT_DIR "/usr/local/opt/openssl")
set (OPENSSL_ROOT_DIR "/usr/local/opt/openssl" CACHE INTERNAL "")
# https://rt.openssl.org/Ticket/Display.html?user=guest&pass=guest&id=2232
if (USE_STATIC_LIBRARIES)
message(WARNING "Disable USE_STATIC_LIBRARIES if you have linking problems with OpenSSL on MacOS")

View File

@ -4,6 +4,7 @@
# include (cmake/limit_jobs.cmake)
cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd
cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES)
option(PARALLEL_COMPILE_JOBS "Define the maximum number of concurrent compilation jobs" "")
if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY)
@ -12,7 +13,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_COMPILER_MEM
set (PARALLEL_COMPILE_JOBS 1)
endif ()
endif ()
if (PARALLEL_COMPILE_JOBS)
if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES))
set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR})
string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE})
set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS})
@ -25,13 +26,12 @@ if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY)
set (PARALLEL_LINK_JOBS 1)
endif ()
endif ()
if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS)
message(STATUS "${CMAKE_CURRENT_SOURCE_DIR}: 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)
if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES))
set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR})
string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK})
set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS})
endif ()
if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS)
message(STATUS "${CMAKE_CURRENT_SOURCE_DIR}: Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}")
endif ()

View File

@ -139,6 +139,7 @@ if (USE_INTERNAL_CAPNP_LIBRARY)
endif ()
if (USE_INTERNAL_POCO_LIBRARY)
set (POCO_VERBOSE_MESSAGES 0 CACHE INTERNAL "")
set (save_CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS})
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
set (_save ${ENABLE_TESTS})

View File

@ -2,13 +2,17 @@ if (USE_INCLUDE_WHAT_YOU_USE)
set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH})
endif ()
set (MAX_COMPILER_MEMORY 2500 CACHE INTERNAL "")
if (MAKE_STATIC_LIBRARIES)
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
if(COMPILER_PIPE)
set(MAX_COMPILER_MEMORY 2500)
else()
set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "")
endif ()
include (../cmake/limit_jobs.cmake)
set(MAX_COMPILER_MEMORY 1500)
endif()
if(MAKE_STATIC_LIBRARIES)
set(MAX_LINKER_MEMORY 3500)
else()
set(MAX_LINKER_MEMORY 2500)
endif()
include(../cmake/limit_jobs.cmake)
include(cmake/find_vectorclass.cmake)

View File

@ -1,5 +1,5 @@
add_library (clickhouse-client-lib ${LINK_MODE} Client.cpp)
target_link_libraries (clickhouse-client-lib PRIVATE clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY})
target_link_libraries (clickhouse-client-lib PRIVATE clickhouse_common_config clickhouse_functions clickhouse_aggregate_functions clickhouse_common_io ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY})
if (READLINE_INCLUDE_DIR)
target_include_directories (clickhouse-client-lib SYSTEM PRIVATE ${READLINE_INCLUDE_DIR})
endif ()

View File

@ -1997,7 +1997,7 @@ protected:
};
{
ThreadPool thread_pool(std::min(num_shards, UInt64(getNumberOfPhysicalCPUCores())));
ThreadPool thread_pool(std::min<UInt64>(num_shards, getNumberOfPhysicalCPUCores()));
for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index)
thread_pool.schedule([=] { do_for_shard(shard_index); });

View File

@ -9,7 +9,7 @@ add_library (clickhouse-odbc-bridge-lib ${LINK_MODE}
validateODBCConnectionString.cpp
)
target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE clickhouse_common_io daemon dbms)
target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE clickhouse_dictionaries daemon dbms clickhouse_common_io)
target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include)
if (USE_POCO_SQLODBC)

View File

@ -411,7 +411,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->setMarkCache(mark_cache_size);
#if USE_EMBEDDED_COMPILER
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", std::numeric_limits<UInt64>::max());
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", 500);
if (compiled_expression_cache_size)
global_context->setCompiledExpressionCache(compiled_expression_cache_size);
#endif

View File

@ -1,4 +1,4 @@
#include <TableFunctions/parseRemoteDescription.h>
#include "parseRemoteDescription.h"
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>

View File

@ -20,9 +20,7 @@ endif()
if(USE_POCO_SQLODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR})
endif()
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR})
endif()
if(Poco_Data_FOUND)
@ -31,9 +29,7 @@ endif()
if(USE_POCO_DATAODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_DataODBC_LIBRARY} ${Poco_Data_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR})
endif()
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR})
endif()
if(USE_POCO_MONGODB)

View File

@ -47,9 +47,11 @@ if (ENABLE_TESTS)
endif ()
if (USE_EMBEDDED_COMPILER)
target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS})
llvm_libs_all(REQUIRED_LLVM_LIBRARIES)
target_link_libraries(clickhouse_functions PRIVATE ${REQUIRED_LLVM_LIBRARIES})
target_include_directories(clickhouse_functions SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS})
endif ()
if (USE_BASE64)
target_include_directories (clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR})
if(USE_BASE64)
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR})
endif()

View File

@ -137,10 +137,7 @@ void AsynchronousMetrics::update()
#if USE_EMBEDDED_COMPILER
{
if (auto compiled_expression_cache = context.getCompiledExpressionCache())
{
set("CompiledExpressionCacheBytes", compiled_expression_cache->weight());
set("CompiledExpressionCacheCount", compiled_expression_cache->count());
}
}
#endif

View File

@ -16,18 +16,21 @@ struct ColumnNamesContext
{
struct JoinedTable
{
const ASTTableExpression * expr;
const ASTTableJoin * join;
const ASTTableExpression * expr = nullptr;
const ASTTableJoin * join = nullptr;
std::optional<String> alias() const
{
String alias;
if (expr->database_and_table_name)
alias = expr->database_and_table_name->tryGetAlias();
else if (expr->table_function)
alias = expr->table_function->tryGetAlias();
else if (expr->subquery)
alias = expr->subquery->tryGetAlias();
if (expr)
{
if (expr->database_and_table_name)
alias = expr->database_and_table_name->tryGetAlias();
else if (expr->table_function)
alias = expr->table_function->tryGetAlias();
else if (expr->subquery)
alias = expr->subquery->tryGetAlias();
}
if (!alias.empty())
return alias;
return {};
@ -35,9 +38,10 @@ struct ColumnNamesContext
std::optional<String> name() const
{
if (auto * node = expr->database_and_table_name.get())
if (auto * identifier = typeid_cast<const ASTIdentifier *>(node))
return identifier->name;
if (expr)
if (auto * node = expr->database_and_table_name.get())
if (auto * identifier = typeid_cast<const ASTIdentifier *>(node))
return identifier->name;
return {};
}

View File

@ -189,70 +189,36 @@ auto wrapJITSymbolResolver(llvm::JITSymbolResolver & jsr)
}
#endif
#if LLVM_VERSION_MAJOR >= 6
struct CountingMMapper final : public llvm::SectionMemoryManager::MemoryMapper
{
MemoryTracker memory_tracker{VariableContext::Global};
llvm::sys::MemoryBlock allocateMappedMemory(llvm::SectionMemoryManager::AllocationPurpose /*purpose*/,
size_t num_bytes,
const llvm::sys::MemoryBlock * const near_block,
unsigned flags,
std::error_code & error_code) override
{
memory_tracker.alloc(num_bytes);
return llvm::sys::Memory::allocateMappedMemory(num_bytes, near_block, flags, error_code);
}
std::error_code protectMappedMemory(const llvm::sys::MemoryBlock & block, unsigned flags) override
{
return llvm::sys::Memory::protectMappedMemory(block, flags);
}
std::error_code releaseMappedMemory(llvm::sys::MemoryBlock & block) override
{
memory_tracker.free(block.size());
return llvm::sys::Memory::releaseMappedMemory(block);
}
};
#if LLVM_VERSION_MAJOR >= 7
using ModulePtr = std::unique_ptr<llvm::Module>;
#else
using ModulePtr = std::shared_ptr<llvm::Module>;
#endif
struct LLVMContext
{
static inline std::atomic<size_t> id_counter{0};
llvm::LLVMContext context;
std::shared_ptr<llvm::LLVMContext> context;
#if LLVM_VERSION_MAJOR >= 7
llvm::orc::ExecutionSession execution_session;
std::unique_ptr<llvm::Module> module;
#else
std::shared_ptr<llvm::Module> module;
#endif
ModulePtr module;
std::unique_ptr<llvm::TargetMachine> machine;
#if LLVM_VERSION_MAJOR >= 6
std::unique_ptr<CountingMMapper> memory_mapper;
#endif
std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
llvm::orc::RTDyldObjectLinkingLayer object_layer;
llvm::orc::IRCompileLayer<decltype(object_layer), llvm::orc::SimpleCompiler> compile_layer;
llvm::DataLayout layout;
llvm::IRBuilder<> builder;
std::unordered_map<std::string, void *> symbols;
size_t id;
LLVMContext()
: context(std::make_shared<llvm::LLVMContext>())
#if LLVM_VERSION_MAJOR >= 7
: module(std::make_unique<llvm::Module>("jit", context))
, module(std::make_unique<llvm::Module>("jit", *context))
#else
: module(std::make_shared<llvm::Module>("jit", context))
, module(std::make_shared<llvm::Module>("jit", *context))
#endif
, machine(getNativeMachine())
#if LLVM_VERSION_MAJOR >= 6
, memory_mapper(std::make_unique<CountingMMapper>())
, memory_manager(std::make_shared<llvm::SectionMemoryManager>(memory_mapper.get()))
#else
, memory_manager(std::make_shared<llvm::SectionMemoryManager>())
#endif
#if LLVM_VERSION_MAJOR >= 7
, object_layer(execution_session, [this](llvm::orc::VModuleKey)
{
@ -263,18 +229,17 @@ struct LLVMContext
#endif
, compile_layer(object_layer, llvm::orc::SimpleCompiler(*machine))
, layout(machine->createDataLayout())
, builder(context)
, id(id_counter++)
, builder(*context)
{
module->setDataLayout(layout);
module->setTargetTriple(machine->getTargetTriple().getTriple());
}
/// returns used memory
size_t compileAllFunctionsToNativeCode()
void compileAllFunctionsToNativeCode()
{
if (!module->size())
return 0;
return;
llvm::PassManagerBuilder pass_manager_builder;
llvm::legacy::PassManager mpm;
llvm::legacy::FunctionPassManager fpm(module.get());
@ -323,26 +288,20 @@ struct LLVMContext
throw Exception("Function " + name + " failed to link", ErrorCodes::CANNOT_COMPILE_CODE);
symbols[name] = reinterpret_cast<void *>(*address);
}
#if LLVM_VERSION_MAJOR >= 6
return memory_mapper->memory_tracker.get();
#else
return 0;
#endif
}
};
class LLVMPreparedFunction : public PreparedFunctionImpl
{
std::string name;
std::shared_ptr<LLVMContext> context;
void * function;
public:
LLVMPreparedFunction(std::string name_, std::shared_ptr<LLVMContext> context)
: name(std::move(name_)), context(context)
LLVMPreparedFunction(const std::string & name_, const std::unordered_map<std::string, void *> & symbols)
: name(name_)
{
auto it = context->symbols.find(name);
if (context->symbols.end() == it)
auto it = symbols.find(name);
if (symbols.end() == it)
throw Exception("Cannot find symbol " + name + " in LLVMContext", ErrorCodes::LOGICAL_ERROR);
function = it->second;
}
@ -373,16 +332,16 @@ public:
}
};
static void compileFunctionToLLVMByteCode(std::shared_ptr<LLVMContext> & context, const IFunctionBase & f)
static void compileFunctionToLLVMByteCode(LLVMContext & context, const IFunctionBase & f)
{
ProfileEvents::increment(ProfileEvents::CompileFunction);
auto & arg_types = f.getArgumentTypes();
auto & b = context->builder;
auto & b = context.builder;
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy(), size_type);
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), context->module.get());
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), context.module.get());
auto args = func->args().begin();
llvm::Value * counter_arg = &*args++;
llvm::Value * columns_arg = &*args++;
@ -504,12 +463,21 @@ static CompilableExpression subexpression(const IFunctionBase & f, std::vector<C
};
}
LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shared_ptr<LLVMContext> context, const Block & sample_block)
: name(actions.back().result_name), context(context)
struct LLVMModuleState
{
std::unordered_map<std::string, void *> symbols;
std::shared_ptr<llvm::LLVMContext> major_context;
std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
};
LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const Block & sample_block)
: name(actions.back().result_name)
, module_state(std::make_unique<LLVMModuleState>())
{
LLVMContext context;
for (const auto & c : sample_block)
/// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)`
if (c.column && getNativeValue(toNativeType(context->builder, c.type), *c.column, 0))
if (c.column && getNativeValue(toNativeType(context.builder, c.type), *c.column, 0))
subexpressions[c.name] = subexpression(c.column, c.type);
for (const auto & action : actions)
{
@ -530,6 +498,11 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shar
originals.push_back(action.function_base);
}
compileFunctionToLLVMByteCode(context, *this);
context.compileAllFunctionsToNativeCode();
module_state->symbols = context.symbols;
module_state->major_context = context.context;
module_state->memory_manager = context.memory_manager;
}
llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const
@ -540,8 +513,7 @@ llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlacehol
return it->second(builder, values);
}
PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared<LLVMPreparedFunction>(name, context); }
PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared<LLVMPreparedFunction>(name, module_state->symbols); }
bool LLVMFunction::isDeterministic() const
{
@ -622,28 +594,6 @@ static bool isCompilable(const IFunctionBase & function)
return function.isCompilable();
}
size_t CompiledExpressionCache::weight() const
{
#if LLVM_VERSION_MAJOR >= 6
std::lock_guard lock(mutex);
size_t result{0};
std::unordered_set<size_t> seen;
for (const auto & cell : cells)
{
auto function_context = cell.second.value->getContext();
if (!seen.count(function_context->id))
{
result += function_context->memory_mapper->memory_tracker.get();
seen.insert(function_context->id);
}
}
return result;
#else
return Base::weight();
#endif
}
std::vector<std::unordered_set<std::optional<size_t>>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns)
{
/// an empty optional is a poisoned value prohibiting the column's producer from being removed
@ -748,21 +698,16 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] ()
{
Stopwatch watch;
std::shared_ptr<LLVMContext> context = std::make_shared<LLVMContext>();
auto result_fn = std::make_shared<LLVMFunction>(inlined_func, context, sample_block);
size_t used_memory = context->compileAllFunctionsToNativeCode();
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory);
std::shared_ptr<LLVMFunction> result_fn;
result_fn = std::make_shared<LLVMFunction>(inlined_func, sample_block);
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
return result_fn;
});
}
else
{
std::shared_ptr<LLVMContext> context = std::make_shared<LLVMContext>();
Stopwatch watch;
fn = std::make_shared<LLVMFunction>(fused[i], context, sample_block);
size_t used_memory = context->compileAllFunctionsToNativeCode();
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory);
fn = std::make_shared<LLVMFunction>(fused[i], sample_block);
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
}

View File

@ -14,19 +14,23 @@
namespace DB
{
struct LLVMContext;
using CompilableExpression = std::function<llvm::Value * (llvm::IRBuilderBase &, const ValuePlaceholders &)>;
struct LLVMModuleState;
class LLVMFunction : public IFunctionBase
{
std::string name;
Names arg_names;
DataTypes arg_types;
std::shared_ptr<LLVMContext> context;
std::vector<FunctionBasePtr> originals;
std::unordered_map<StringRef, CompilableExpression> subexpressions;
std::unique_ptr<LLVMModuleState> module_state;
public:
LLVMFunction(const ExpressionActions::Actions & actions, std::shared_ptr<LLVMContext> context, const Block & sample_block);
LLVMFunction(const ExpressionActions::Actions & actions, const Block & sample_block);
bool isCompilable() const override { return true; }
@ -54,8 +58,7 @@ public:
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override;
std::shared_ptr<LLVMContext> getContext() const { return context; }
const LLVMModuleState * getLLVMModuleState() const { return module_state.get(); }
};
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
@ -63,13 +66,9 @@ public:
*/
class CompiledExpressionCache : public LRUCache<UInt128, LLVMFunction, UInt128Hash>
{
private:
using Base = LRUCache<UInt128, LLVMFunction, UInt128Hash>;
public:
using Base = LRUCache<UInt128, LLVMFunction, UInt128Hash>;
using Base::Base;
size_t weight() const;
};
/// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable

View File

@ -15,7 +15,7 @@
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <Poco/Path.h>
#include <TableFunctions/parseRemoteDescription.h>
#include <Common/parseRemoteDescription.h>
#include <Common/typeid_cast.h>

View File

@ -1,3 +1,5 @@
#include "TableFunctionRemote.h"
#include <Storages/getStructureOfRemoteTable.h>
#include <Storages/StorageDistributed.h>
#include <Parsers/ASTIdentifier.h>
@ -8,10 +10,8 @@
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
#include <TableFunctions/TableFunctionRemote.h>
#include <Common/parseRemoteDescription.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/parseRemoteDescription.h>
namespace DB

View File

@ -2,7 +2,7 @@
Array of `T`-type items.
`T` can be anything, including an array. Use multi-dimensional arrays with caution. ClickHouse has limited support for multi-dimensional arrays. For example, they can't be stored in `MergeTree` tables.
`T` can be anything, including an array.
## Creating an array

View File

@ -99,7 +99,7 @@ def build_for_lang(lang, args):
site_dir=os.path.join(args.output_dir, lang),
strict=True,
theme=theme_cfg,
copyright='©20162018 Yandex LLC',
copyright='©20162019 Yandex LLC',
use_directory_urls=True,
repo_name='yandex/ClickHouse',
repo_url='https://github.com/yandex/ClickHouse/',

View File

@ -7,6 +7,7 @@ add_executable (date_lut4 date_lut4.cpp)
add_executable (date_lut_default_timezone date_lut_default_timezone.cpp)
add_executable (multi_version multi_version.cpp)
add_executable (local_date_time_comparison local_date_time_comparison.cpp)
add_executable (realloc-perf allocator.cpp)
set(PLATFORM_LIBS ${CMAKE_DL_LIBS})
@ -17,6 +18,7 @@ target_link_libraries (date_lut4 common ${PLATFORM_LIBS})
target_link_libraries (date_lut_default_timezone common ${PLATFORM_LIBS})
target_link_libraries (multi_version common)
target_link_libraries (local_date_time_comparison common)
target_link_libraries (realloc-perf common)
add_check(multi_version)
add_check(local_date_time_comparison)

View File

@ -0,0 +1,47 @@
#include <cstdlib>
#include <cstring>
#include <vector>
#include <thread>
void thread_func()
{
for (size_t i = 0; i < 100; ++i)
{
size_t size = 4096;
void * buf = malloc(size);
if (!buf)
abort();
memset(buf, 0, size);
while (size < 1048576)
{
size_t next_size = size * 4;
void * new_buf = realloc(buf, next_size);
if (!new_buf)
abort();
buf = new_buf;
memset(reinterpret_cast<char*>(buf) + size, 0, next_size - size);
size = next_size;
}
free(buf);
}
}
int main(int, char **)
{
std::vector<std::thread> threads(16);
for (size_t i = 0; i < 1000; ++i)
{
for (auto & thread : threads)
thread = std::thread(thread_func);
for (auto & thread : threads)
thread.join();
}
return 0;
}

View File

@ -3,12 +3,12 @@ if (NOT NO_WERROR)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror")
endif ()
if (MAKE_STATIC_LIBRARIES)
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
if(MAKE_STATIC_LIBRARIES)
set(MAX_LINKER_MEMORY 3500)
else()
set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "")
endif ()
include (../cmake/limit_jobs.cmake)
set(MAX_LINKER_MEMORY 2500)
endif()
include(../cmake/limit_jobs.cmake)
# Utils used in package
add_subdirectory (config-processor)

View File

@ -22,5 +22,5 @@ env TEST_RUN=1 \
`# Use all possible contrib libs from system` \
`# psmisc - killall` \
`# gdb - symbol test in pbuilder` \
EXTRAPACKAGES="psmisc libboost-program-options-dev libboost-system-dev libboost-filesystem-dev libboost-thread-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libsparsehash-dev librdkafka-dev libpoco-dev unixodbc-dev libsparsehash-dev libgoogle-perftools-dev libzstd-dev libre2-dev libunwind-dev googletest libcctz-dev libcapnp-dev libjemalloc-dev libssl-dev $EXTRAPACKAGES" \
EXTRAPACKAGES="psmisc libboost-program-options-dev libboost-system-dev libboost-filesystem-dev libboost-thread-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libsparsehash-dev librdkafka-dev libpoco-dev unixodbc-dev libsparsehash-dev libgoogle-perftools-dev libzstd-dev libre2-dev libunwind-dev googletest libcctz-dev libcapnp-dev libjemalloc-dev libssl-dev libunwind-dev libxml2-dev libgsasl7-dev $EXTRAPACKAGES" \
pdebuild --configfile $ROOT_DIR/debian/.pbuilderrc $PDEBUILD_OPT

View File

@ -0,0 +1,6 @@
#!/bin/bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CMAKE_FLAGS+=" -DCLICKHOUSE_SPLIT_BINARY=1 "
. $CUR_DIR/build_debian_unbundled.sh