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

This commit is contained in:
BayoNet 2018-11-19 14:54:59 +03:00
commit ef2f533a67
170 changed files with 4308 additions and 5109 deletions

View File

@ -1,3 +1,18 @@
## ClickHouse release 18.14.13, 2018-11-08
### Исправления ошибок:
* Исправлена ошибка `Block structure mismatch in MergingSorted stream`. [#3162](https://github.com/yandex/ClickHouse/issues/3162)
* Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг `<secure>`). [#3465](https://github.com/yandex/ClickHouse/pull/3465)
* Исправлена ошибка при использовании `SAMPLE`, `PREWHERE` и столбцов-алиасов. [#3543](https://github.com/yandex/ClickHouse/pull/3543)
* Исправлена редкая ошибка `unknown compression method` при использовании настройки `min_bytes_to_use_direct_io`. [3544](https://github.com/yandex/ClickHouse/pull/3544)
### Улучшения производительности:
* Исправлена деградация производительности запросов с `GROUP BY` столбцов типа Int16, Date на процессорах AMD EPYC. [Игорь Лапко](https://github.com/yandex/ClickHouse/pull/3512)
* Исправлена деградация производительности при обработке длинных строк. [#3530](https://github.com/yandex/ClickHouse/pull/3530)
### Улучшения процесса сборки ClickHouse:
* Доработки для упрощения сборки в Arcadia. [#3475](https://github.com/yandex/ClickHouse/pull/3475), [#3535](https://github.com/yandex/ClickHouse/pull/3535)
## ClickHouse release 18.14.12, 2018-11-02
### Исправления ошибок:

View File

@ -115,8 +115,8 @@ endif ()
include (cmake/test_cpu.cmake)
option (ARCHNATIVE "Enable -march=native compiler flag" OFF)
if (ARCHNATIVE)
option (ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM})
if (ARCH_NATIVE)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native")
endif ()
@ -155,6 +155,7 @@ if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
endif ()
set(THREADS_PREFER_PTHREAD_FLAG ON)
find_package (Threads)
include (cmake/test_compiler.cmake)

View File

@ -18,12 +18,12 @@ if (ENABLE_EMBEDDED_COMPILER)
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
find_package(LLVM ${CMAKE_CXX_COMPILER_VERSION} CONFIG PATHS ${LLVM_PATHS})
else ()
find_package (LLVM 6 CONFIG PATHS ${LLVM_PATHS})
find_package (LLVM 7 CONFIG PATHS ${LLVM_PATHS})
if (NOT LLVM_FOUND)
find_package (LLVM 5 CONFIG PATHS ${LLVM_PATHS})
find_package (LLVM 6 CONFIG PATHS ${LLVM_PATHS})
endif ()
if (NOT LLVM_FOUND)
find_package (LLVM 7 CONFIG PATHS ${LLVM_PATHS})
find_package (LLVM 5 CONFIG PATHS ${LLVM_PATHS})
endif ()
endif ()

View File

@ -1,4 +1,6 @@
if (NOT ARCH_ARM)
option (ENABLE_RDKAFKA "Enable kafka" ON)
endif ()
if (ENABLE_RDKAFKA)

View File

@ -14,6 +14,7 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
endif ()
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_LIBRARIES ${CMAKE_REQUIRED_LIBRARIES} Threads::Threads)
check_cxx_source_compiles("
#include <iostream>
@ -25,8 +26,6 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
else ()
cmake_push_check_state ()
set (TEST_FLAG "-no-pie")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
@ -40,7 +39,6 @@ else ()
set (FLAG_NO_PIE ${TEST_FLAG})
endif ()
endif ()
cmake_pop_check_state ()

View File

@ -55,7 +55,7 @@ endif ()
if (USE_INTERNAL_ZLIB_LIBRARY)
set (ZLIB_COMPAT 1 CACHE INTERNAL "") # also enables WITH_GZFILEOP
set (WITH_NATIVE_INSTRUCTIONS ${ARCHNATIVE} CACHE INTERNAL "")
set (WITH_NATIVE_INSTRUCTIONS ${ARCH_NATIVE} CACHE INTERNAL "")
if (OS_FREEBSD OR ARCH_I386)
set (WITH_OPTIM 0 CACHE INTERNAL "") # Bug in assembler
endif ()
@ -159,14 +159,6 @@ if (USE_INTERNAL_POCO_LIBRARY)
endif ()
endif ()
if (USE_INTERNAL_LLVM_LIBRARY)
# ld: unknown option: --color-diagnostics
if (APPLE)
set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "")
endif ()
add_subdirectory (llvm/llvm)
endif ()
if (USE_INTERNAL_GTEST_LIBRARY)
# Google Test from sources
add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest)
@ -174,3 +166,28 @@ if (USE_INTERNAL_GTEST_LIBRARY)
target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0)
target_include_directories (gtest SYSTEM INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/googletest/include)
endif ()
if (USE_INTERNAL_LLVM_LIBRARY)
file(GENERATE OUTPUT ${CMAKE_CURRENT_BINARY_DIR}/empty.cpp CONTENT " ")
add_library(LLVM0 ${CMAKE_CURRENT_BINARY_DIR}/empty.cpp) # silly cmake bug fix
# ld: unknown option: --color-diagnostics
if (APPLE)
set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "")
endif ()
set (LLVM_ENABLE_EH 1 CACHE INTERNAL "")
set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "")
set (LLVM_INCLUDE_TESTS 0 CACHE INTERNAL "")
set (LLVM_INCLUDE_EXAMPLES 0 CACHE INTERNAL "")
set (LLVM_INCLUDE_TOOLS 0 CACHE INTERNAL "")
set (LLVM_INSTALL_TOOLCHAIN_ONLY 0 CACHE INTERNAL "")
set (CLANG_BUILT_STANDALONE 0 CACHE INTERNAL "")
set (LLDB_BUILT_STANDALONE 0 CACHE INTERNAL "")
set (CLANG_ENABLE_STATIC_ANALYZER 0 CACHE INTERNAL "")
set (CLANG_ENABLE_ARCMT 0 CACHE INTERNAL "")
set (CLANG_BUILD_TOOLS 0 CACHE INTERNAL "")
set (BENCHMARK_ENABLE_GTEST_TESTS 0 CACHE INTERNAL "")
set (BENCHMARK_ENABLE_ASSEMBLY_TESTS 0 CACHE INTERNAL "")
set (LLVM_TARGETS_TO_BUILD "X86;AArch64" CACHE INTERNAL "")
add_subdirectory (llvm/llvm)
endif ()

2
contrib/jemalloc vendored

@ -1 +1 @@
Subproject commit 41b7372eadee941b9164751b8d4963f915d3ceae
Subproject commit cd2931ad9bbd78208565716ab102e86d858c2fff

View File

@ -48,8 +48,6 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
add_definitions ("-fno-tree-loop-distribute-patterns")
endif ()
find_package (Threads)
add_subdirectory (src)
set(dbms_headers)
@ -61,7 +59,6 @@ add_headers_and_sources(clickhouse_common_io src/Common)
add_headers_and_sources(clickhouse_common_io src/Common/HashTable)
add_headers_and_sources(clickhouse_common_io src/IO)
add_headers_and_sources(dbms src/Analyzers)
add_headers_and_sources(dbms src/Core)
add_headers_and_sources(dbms src/DataStreams)
add_headers_and_sources(dbms src/DataTypes)

View File

@ -1,5 +1,7 @@
add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG)
link_directories(${LLVM_LIBRARY_DIRS})
add_library(clickhouse-compiler-lib ${LINK_MODE}
driver.cpp
cc1_main.cpp

View File

@ -1,5 +1,7 @@
add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG)
link_directories(${LLVM_LIBRARY_DIRS})
add_library(clickhouse-compiler-lib ${LINK_MODE}
driver.cpp
cc1_main.cpp

View File

@ -441,7 +441,7 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
auto Pair = StringRef(S).split('=');
auto Sym = Pair.first;
auto Val = Pair.second;
int64_t Value;
int64_t Value = 0;
// We have already error checked this in the driver.
Val.getAsInteger(0, Value);
Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value);

View File

@ -1,8 +1,11 @@
add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG)
link_directories(${LLVM_LIBRARY_DIRS})
add_library(clickhouse-compiler-lib ${LINK_MODE}
driver.cpp
cc1_main.cpp
cc1gen_reproducer_main.cpp
cc1as_main.cpp
lld.cpp)
@ -19,7 +22,6 @@ target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE
# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory.
target_link_libraries(clickhouse-compiler-lib PRIVATE
clangBasic clangCodeGen clangDriver
clangFrontend
clangFrontendTool

View File

@ -16,6 +16,7 @@
#include "llvm/Option/Arg.h"
#include "clang/CodeGen/ObjectFilePCHContainerOperations.h"
#include "clang/Config/config.h"
#include "clang/Basic/Stack.h"
#include "clang/Driver/DriverDiagnostic.h"
#include "clang/Driver/Options.h"
#include "clang/Frontend/CompilerInstance.h"
@ -76,13 +77,6 @@ void initializePollyPasses(llvm::PassRegistry &Registry);
#endif
#ifdef CLANG_HAVE_RLIMITS
// The amount of stack we think is "sufficient". If less than this much is
// available, we may be unable to reach our template instantiation depth
// limit and other similar limits.
// FIXME: Unify this with the stack we request when spawning a thread to build
// a module.
static const int kSufficientStack = 8 << 20;
#if defined(__linux__) && defined(__PIE__)
static size_t getCurrentStackAllocation() {
// If we can't compute the current stack usage, allow for 512K of command
@ -120,7 +114,7 @@ static size_t getCurrentStackAllocation() {
#include <alloca.h>
LLVM_ATTRIBUTE_NOINLINE
static void ensureStackAddressSpace(int ExtraChunks = 0) {
static void ensureStackAddressSpace() {
// Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary
// relatively close to the stack (they are only guaranteed to be 128MiB
// apart). This results in crashes if we happen to heap-allocate more than
@ -129,7 +123,7 @@ static void ensureStackAddressSpace(int ExtraChunks = 0) {
// To avoid these crashes, ensure that we have sufficient virtual memory
// pages allocated before we start running.
size_t Curr = getCurrentStackAllocation();
const int kTargetStack = kSufficientStack - 256 * 1024;
const int kTargetStack = DesiredStackSize - 256 * 1024;
if (Curr < kTargetStack) {
volatile char *volatile Alloc =
static_cast<volatile char *>(alloca(kTargetStack - Curr));
@ -149,21 +143,23 @@ static void ensureSufficientStack() {
// Increase the soft stack limit to our desired level, if necessary and
// possible.
if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) {
if (rlim.rlim_cur != RLIM_INFINITY &&
rlim.rlim_cur < rlim_t(DesiredStackSize)) {
// Try to allocate sufficient stack.
if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack)
rlim.rlim_cur = kSufficientStack;
if (rlim.rlim_max == RLIM_INFINITY ||
rlim.rlim_max >= rlim_t(DesiredStackSize))
rlim.rlim_cur = DesiredStackSize;
else if (rlim.rlim_cur == rlim.rlim_max)
return;
else
rlim.rlim_cur = rlim.rlim_max;
if (setrlimit(RLIMIT_STACK, &rlim) != 0 ||
rlim.rlim_cur != kSufficientStack)
rlim.rlim_cur != DesiredStackSize)
return;
}
// We should now have a stack of size at least kSufficientStack. Ensure
// We should now have a stack of size at least DesiredStackSize. Ensure
// that we can actually use that much, if necessary.
ensureStackAddressSpace();
}

View File

@ -29,6 +29,7 @@
#include "llvm/MC/MCContext.h"
#include "llvm/MC/MCInstrInfo.h"
#include "llvm/MC/MCObjectFileInfo.h"
#include "llvm/MC/MCObjectWriter.h"
#include "llvm/MC/MCParser/MCAsmParser.h"
#include "llvm/MC/MCParser/MCTargetAsmParser.h"
#include "llvm/MC/MCRegisterInfo.h"
@ -59,10 +60,9 @@ using namespace clang::driver::options;
using namespace llvm;
using namespace llvm::opt;
namespace {
/// \brief Helper class for representing a single invocation of the assembler.
/// Helper class for representing a single invocation of the assembler.
struct AssemblerInvocation {
/// @name Target Options
/// @{
@ -94,9 +94,11 @@ struct AssemblerInvocation {
std::string DwarfDebugFlags;
std::string DwarfDebugProducer;
std::string DebugCompilationDir;
std::map<const std::string, const std::string> DebugPrefixMap;
llvm::DebugCompressionType CompressDebugSections =
llvm::DebugCompressionType::None;
std::string MainFileName;
std::string SplitDwarfFile;
/// @}
/// @name Frontend Options
@ -232,6 +234,9 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts,
Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir);
Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name);
for (const auto &Arg : Args.getAllArgValues(OPT_fdebug_prefix_map_EQ))
Opts.DebugPrefixMap.insert(StringRef(Arg).split('='));
// Frontend Options
if (Args.hasArg(OPT_INPUT)) {
bool First = true;
@ -247,6 +252,7 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts,
}
Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm);
Opts.OutputPath = Args.getLastArgValue(OPT_o);
Opts.SplitDwarfFile = Args.getLastArgValue(OPT_split_dwarf_file);
if (Arg *A = Args.getLastArg(OPT_filetype)) {
StringRef Name = A->getValue();
unsigned OutputType = StringSwitch<unsigned>(Name)
@ -282,22 +288,17 @@ bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts,
}
static std::unique_ptr<raw_fd_ostream>
getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags,
bool Binary) {
if (Opts.OutputPath.empty())
Opts.OutputPath = "-";
getOutputStream(StringRef Path, DiagnosticsEngine &Diags, bool Binary) {
// Make sure that the Out file gets unlinked from the disk if we get a
// SIGINT.
if (Opts.OutputPath != "-")
sys::RemoveFileOnSignal(Opts.OutputPath);
if (Path != "-")
sys::RemoveFileOnSignal(Path);
std::error_code EC;
auto Out = llvm::make_unique<raw_fd_ostream>(
Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text));
Path, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text));
if (EC) {
Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath
<< EC.message();
Diags.Report(diag::err_fe_unable_to_open_output) << Path << EC.message();
return nullptr;
}
@ -342,9 +343,15 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations);
bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj;
std::unique_ptr<raw_fd_ostream> FDOS = getOutputStream(Opts, Diags, IsBinary);
if (Opts.OutputPath.empty())
Opts.OutputPath = "-";
std::unique_ptr<raw_fd_ostream> FDOS =
getOutputStream(Opts.OutputPath, Diags, IsBinary);
if (!FDOS)
return true;
std::unique_ptr<raw_fd_ostream> DwoOS;
if (!Opts.SplitDwarfFile.empty())
DwoOS = getOutputStream(Opts.SplitDwarfFile, Diags, IsBinary);
// FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and
// MCObjectFileInfo needs a MCContext reference in order to initialize itself.
@ -374,6 +381,9 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer));
if (!Opts.DebugCompilationDir.empty())
Ctx.setCompilationDir(Opts.DebugCompilationDir);
if (!Opts.DebugPrefixMap.empty())
for (const auto &KV : Opts.DebugPrefixMap)
Ctx.addDebugPrefixMapEntry(KV.first, KV.second);
if (!Opts.MainFileName.empty())
Ctx.setMainFileName(StringRef(Opts.MainFileName));
Ctx.setDwarfVersion(Opts.DwarfVersion);
@ -427,11 +437,14 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
MCTargetOptions MCOptions;
std::unique_ptr<MCAsmBackend> MAB(
TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions));
std::unique_ptr<MCObjectWriter> OW =
DwoOS ? MAB->createDwoObjectWriter(*Out, *DwoOS)
: MAB->createObjectWriter(*Out);
Triple T(Opts.Triple);
Str.reset(TheTarget->createMCObjectStreamer(
T, Ctx, std::move(MAB), *Out, std::move(CE), *STI, Opts.RelaxAll,
Opts.IncrementalLinkerCompatible,
T, Ctx, std::move(MAB), std::move(OW), std::move(CE), *STI,
Opts.RelaxAll, Opts.IncrementalLinkerCompatible,
/*DWARFMustBeAtTheEnd*/ true));
Str.get()->InitSections(Opts.NoExecStack);
}
@ -456,7 +469,7 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
auto Pair = StringRef(S).split('=');
auto Sym = Pair.first;
auto Val = Pair.second;
int64_t Value;
int64_t Value = 1;
// We have already error checked this in the driver.
Val.getAsInteger(0, Value);
Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value);
@ -475,14 +488,18 @@ static bool ExecuteAssembler(AssemblerInvocation &Opts,
FDOS.reset();
// Delete output file if there were errors.
if (Failed && Opts.OutputPath != "-")
if (Failed) {
if (Opts.OutputPath != "-")
sys::fs::remove(Opts.OutputPath);
if (!Opts.SplitDwarfFile.empty() && Opts.SplitDwarfFile != "-")
sys::fs::remove(Opts.SplitDwarfFile);
}
return Failed;
}
static void LLVMErrorHandler(void *UserData, const std::string &Message,
bool /*GenCrashDiag*/) {
bool GenCrashDiag) {
DiagnosticsEngine &Diags = *static_cast<DiagnosticsEngine*>(UserData);
Diags.Report(diag::err_fe_error_backend) << Message;
@ -491,7 +508,7 @@ static void LLVMErrorHandler(void *UserData, const std::string &Message,
exit(1);
}
int cc1as_main(ArrayRef<const char *> Argv, const char */*Argv0*/, void */*MainAddr*/) {
int cc1as_main(ArrayRef<const char *> Argv, const char *Argv0, void *MainAddr) {
// Initialize targets and assembly printers/parsers.
InitializeAllTargetInfos();
InitializeAllTargetMCs();

View File

@ -0,0 +1,196 @@
//===-- cc1gen_reproducer_main.cpp - Clang reproducer generator ----------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
// This is the entry point to the clang -cc1gen-reproducer functionality, which
// generates reproducers for invocations for clang-based tools.
//
//===----------------------------------------------------------------------===//
#include "clang/Basic/Diagnostic.h"
#include "clang/Basic/LLVM.h"
#include "clang/Basic/VirtualFileSystem.h"
#include "clang/Driver/Compilation.h"
#include "clang/Driver/Driver.h"
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ADT/STLExtras.h"
#include "llvm/Support/FileSystem.h"
#include "llvm/Support/TargetSelect.h"
#include "llvm/Support/YAMLTraits.h"
#include "llvm/Support/raw_ostream.h"
using namespace clang;
namespace {
struct UnsavedFileHash {
std::string Name;
std::string MD5;
};
struct ClangInvocationInfo {
std::string Toolchain;
std::string LibclangOperation;
std::string LibclangOptions;
std::vector<std::string> Arguments;
std::vector<std::string> InvocationArguments;
std::vector<UnsavedFileHash> UnsavedFileHashes;
bool Dump = false;
};
} // end anonymous namespace
LLVM_YAML_IS_SEQUENCE_VECTOR(UnsavedFileHash)
namespace llvm {
namespace yaml {
template <> struct MappingTraits<UnsavedFileHash> {
static void mapping(IO &IO, UnsavedFileHash &Info) {
IO.mapRequired("name", Info.Name);
IO.mapRequired("md5", Info.MD5);
}
};
template <> struct MappingTraits<ClangInvocationInfo> {
static void mapping(IO &IO, ClangInvocationInfo &Info) {
IO.mapRequired("toolchain", Info.Toolchain);
IO.mapOptional("libclang.operation", Info.LibclangOperation);
IO.mapOptional("libclang.opts", Info.LibclangOptions);
IO.mapRequired("args", Info.Arguments);
IO.mapOptional("invocation-args", Info.InvocationArguments);
IO.mapOptional("unsaved_file_hashes", Info.UnsavedFileHashes);
}
};
} // end namespace yaml
} // end namespace llvm
static std::string generateReproducerMetaInfo(const ClangInvocationInfo &Info) {
std::string Result;
llvm::raw_string_ostream OS(Result);
OS << '{';
bool NeedComma = false;
auto EmitKey = [&](StringRef Key) {
if (NeedComma)
OS << ", ";
NeedComma = true;
OS << '"' << Key << "\": ";
};
auto EmitStringKey = [&](StringRef Key, StringRef Value) {
if (Value.empty())
return;
EmitKey(Key);
OS << '"' << Value << '"';
};
EmitStringKey("libclang.operation", Info.LibclangOperation);
EmitStringKey("libclang.opts", Info.LibclangOptions);
if (!Info.InvocationArguments.empty()) {
EmitKey("invocation-args");
OS << '[';
for (const auto &Arg : llvm::enumerate(Info.InvocationArguments)) {
if (Arg.index())
OS << ',';
OS << '"' << Arg.value() << '"';
}
OS << ']';
}
OS << '}';
// FIXME: Compare unsaved file hashes and report mismatch in the reproducer.
if (Info.Dump)
llvm::outs() << "REPRODUCER METAINFO: " << OS.str() << "\n";
return std::move(OS.str());
}
/// Generates a reproducer for a set of arguments from a specific invocation.
static llvm::Optional<driver::Driver::CompilationDiagnosticReport>
generateReproducerForInvocationArguments(ArrayRef<const char *> Argv,
const ClangInvocationInfo &Info) {
using namespace driver;
auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(Argv[0]);
IntrusiveRefCntPtr<DiagnosticOptions> DiagOpts = new DiagnosticOptions;
IntrusiveRefCntPtr<DiagnosticIDs> DiagID(new DiagnosticIDs());
DiagnosticsEngine Diags(DiagID, &*DiagOpts, new IgnoringDiagConsumer());
ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false);
Driver TheDriver(Argv[0], llvm::sys::getDefaultTargetTriple(), Diags);
TheDriver.setTargetAndMode(TargetAndMode);
std::unique_ptr<Compilation> C(TheDriver.BuildCompilation(Argv));
if (C && !C->containsError()) {
for (const auto &J : C->getJobs()) {
if (const Command *Cmd = dyn_cast<Command>(&J)) {
Driver::CompilationDiagnosticReport Report;
TheDriver.generateCompilationDiagnostics(
*C, *Cmd, generateReproducerMetaInfo(Info), &Report);
return Report;
}
}
}
return None;
}
std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes);
static void printReproducerInformation(
llvm::raw_ostream &OS, const ClangInvocationInfo &Info,
const driver::Driver::CompilationDiagnosticReport &Report) {
OS << "REPRODUCER:\n";
OS << "{\n";
OS << R"("files":[)";
for (const auto &File : llvm::enumerate(Report.TemporaryFiles)) {
if (File.index())
OS << ',';
OS << '"' << File.value() << '"';
}
OS << "]\n}\n";
}
int cc1gen_reproducer_main(ArrayRef<const char *> Argv, const char *Argv0,
void *MainAddr) {
if (Argv.size() < 1) {
llvm::errs() << "error: missing invocation file\n";
return 1;
}
// Parse the invocation descriptor.
StringRef Input = Argv[0];
llvm::ErrorOr<std::unique_ptr<llvm::MemoryBuffer>> Buffer =
llvm::MemoryBuffer::getFile(Input);
if (!Buffer) {
llvm::errs() << "error: failed to read " << Input << ": "
<< Buffer.getError().message() << "\n";
return 1;
}
llvm::yaml::Input YAML(Buffer.get()->getBuffer());
ClangInvocationInfo InvocationInfo;
YAML >> InvocationInfo;
if (Argv.size() > 1 && Argv[1] == StringRef("-v"))
InvocationInfo.Dump = true;
// Create an invocation that will produce the reproducer.
std::vector<const char *> DriverArgs;
for (const auto &Arg : InvocationInfo.Arguments)
DriverArgs.push_back(Arg.c_str());
std::string Path = GetExecutablePath(Argv0, /*CanonicalPrefixes=*/true);
DriverArgs[0] = Path.c_str();
llvm::Optional<driver::Driver::CompilationDiagnosticReport> Report =
generateReproducerForInvocationArguments(DriverArgs, InvocationInfo);
// Emit the information about the reproduce files to stdout.
int Result = 1;
if (Report) {
printReproducerInformation(llvm::outs(), InvocationInfo, *Report);
Result = 0;
}
// Remove the input file.
llvm::sys::fs::remove(Input);
return Result;
}

View File

@ -310,6 +310,8 @@ static int ExecuteCC1Tool(ArrayRef<const char *> argv, StringRef Tool) {
return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP);
if (Tool == "as")
return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP);
if (Tool == "gen-reproducer")
return cc1gen_reproducer_main(argv.slice(2), argv[0], GetExecutablePathVP);
// Reject unknown tools.
llvm::errs() << "error: unknown integrated tool '" << Tool << "'. "
@ -317,7 +319,7 @@ static int ExecuteCC1Tool(ArrayRef<const char *> argv, StringRef Tool) {
return 1;
}
int mainEntryClickHouseClang(int argc_, char **argv_) {
int mainEntryClickHouseClang(int argc_, /* const */ char **argv_) {
llvm::InitLLVM X(argc_, argv_);
SmallVector<const char *, 256> argv(argv_, argv_ + argc_);

View File

@ -1,10 +1,150 @@
#include "lld/Common/Driver.h"
#include "llvm/Support/InitLLVM.h"
#include <vector>
//===- tools/lld/lld.cpp - Linker Driver Dispatcher -----------------------===//
//
// The LLVM Linker
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
// This file contains the main function of the lld executable. The main
// function is a thin wrapper which dispatches to the platform specific
// driver.
//
// lld is a single executable that contains four different linkers for ELF,
// COFF, WebAssembly and Mach-O. The main function dispatches according to
// argv[0] (i.e. command name). The most common name for each target is shown
// below:
//
// - ld.lld: ELF (Unix)
// - ld64: Mach-O (macOS)
// - lld-link: COFF (Windows)
// - ld-wasm: WebAssembly
//
// lld can be invoked as "lld" along with "-flavor" option. This is for
// backward compatibility and not recommended.
//
//===----------------------------------------------------------------------===//
int mainEntryClickHouseLLD(int argc, char ** argv)
{
llvm::InitLLVM X(argc, argv);
std::vector<const char *> args(argv, argv + argc);
return !lld::elf::link(args, false);
#include "lld/Common/Driver.h"
#include "llvm/ADT/STLExtras.h"
#include "llvm/ADT/StringSwitch.h"
#include "llvm/ADT/Twine.h"
#include "llvm/Support/InitLLVM.h"
#include "llvm/Support/Path.h"
#include <cstdlib>
using namespace lld;
using namespace llvm;
using namespace llvm::sys;
/*
enum Flavor {
Invalid,
Gnu, // -flavor gnu
WinLink, // -flavor link
Darwin, // -flavor darwin
Wasm, // -flavor wasm
};
LLVM_ATTRIBUTE_NORETURN static void die(const Twine &S) {
errs() << S << "\n";
exit(1);
}
static Flavor getFlavor(StringRef S) {
return StringSwitch<Flavor>(S)
.CasesLower("ld", "ld.lld", "gnu", Gnu)
.CasesLower("wasm", "ld-wasm", Wasm)
.CaseLower("link", WinLink)
.CasesLower("ld64", "ld64.lld", "darwin", Darwin)
.Default(Invalid);
}
static bool isPETarget(const std::vector<const char *> &V) {
for (auto It = V.begin(); It + 1 != V.end(); ++It) {
if (StringRef(*It) != "-m")
continue;
StringRef S = *(It + 1);
return S == "i386pe" || S == "i386pep" || S == "thumb2pe" || S == "arm64pe";
}
return false;
}
static Flavor parseProgname(StringRef Progname) {
#if __APPLE__
// Use Darwin driver for "ld" on Darwin.
if (Progname == "ld")
return Darwin;
#endif
#if LLVM_ON_UNIX
// Use GNU driver for "ld" on other Unix-like system.
if (Progname == "ld")
return Gnu;
#endif
// Progname may be something like "lld-gnu". Parse it.
SmallVector<StringRef, 3> V;
Progname.split(V, "-");
for (StringRef S : V)
if (Flavor F = getFlavor(S))
return F;
return Invalid;
}
static Flavor parseFlavor(std::vector<const char *> &V) {
// Parse -flavor option.
if (V.size() > 1 && V[1] == StringRef("-flavor")) {
if (V.size() <= 2)
die("missing arg value for '-flavor'");
Flavor F = getFlavor(V[2]);
if (F == Invalid)
die("Unknown flavor: " + StringRef(V[2]));
V.erase(V.begin() + 1, V.begin() + 3);
return F;
}
// Deduct the flavor from argv[0].
StringRef Arg0 = path::filename(V[0]);
if (Arg0.endswith_lower(".exe"))
Arg0 = Arg0.drop_back(4);
return parseProgname(Arg0);
}
*/
// If this function returns true, lld calls _exit() so that it quickly
// exits without invoking destructors of globally allocated objects.
//
// We don't want to do that if we are running tests though, because
// doing that breaks leak sanitizer. So, lit sets this environment variable,
// and we use it to detect whether we are running tests or not.
static bool canExitEarly() { return StringRef(getenv("LLD_IN_TEST")) != "1"; }
/// Universal linker main(). This linker emulates the gnu, darwin, or
/// windows linker based on the argv[0] or -flavor option.
int mainEntryClickHouseLLD(int Argc, /* const */ char **Argv) {
InitLLVM X(Argc, Argv);
std::vector<const char *> Args(Argv, Argv + Argc);
/*
switch (parseFlavor(Args)) {
case Gnu:
if (isPETarget(Args))
return !mingw::link(Args);
*/
return !elf::link(Args, canExitEarly());
/*
case WinLink:
return !coff::link(Args, canExitEarly());
case Darwin:
return !mach_o::link(Args, canExitEarly());
case Wasm:
return !wasm::link(Args, canExitEarly());
default:
die("lld is a generic driver.\n"
"Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-lld"
" (WebAssembly) instead");
}
*/
}

View File

@ -0,0 +1,49 @@
add_definitions(-Wno-error -Wno-unused-parameter -Wno-non-virtual-dtor -U_LIBCPP_DEBUG)
link_directories(${LLVM_LIBRARY_DIRS})
add_library(clickhouse-compiler-lib ${LINK_MODE}
driver.cpp
cc1_main.cpp
cc1as_main.cpp
lld.cpp)
target_compile_options(clickhouse-compiler-lib PRIVATE -fno-rtti -fno-exceptions -g0)
string(REPLACE "${INCLUDE_DEBUG_HELPERS}" "" CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS}) # cant compile with -fno-rtti
llvm_libs_all(REQUIRED_LLVM_LIBRARIES)
message(STATUS "Using LLVM ${LLVM_VERSION}: ${LLVM_INCLUDE_DIRS} : ${REQUIRED_LLVM_LIBRARIES}")
target_include_directories(clickhouse-compiler-lib SYSTEM PRIVATE ${LLVM_INCLUDE_DIRS})
# This is extracted almost directly from CMakeFiles/.../link.txt in LLVM build directory.
target_link_libraries(clickhouse-compiler-lib PRIVATE
clangBasic clangCodeGen clangDriver
clangFrontend
clangFrontendTool
clangRewriteFrontend clangARCMigrate clangStaticAnalyzerFrontend
clangParse clangSerialization clangSema clangEdit clangStaticAnalyzerCheckers
clangASTMatchers clangStaticAnalyzerCore clangAnalysis clangAST clangRewrite clangLex clangBasic
clangCrossTU clangIndex
lldCOFF
lldDriver
lldELF
lldMinGW
lldMachO
lldReaderWriter
lldYAML
lldCommon
lldCore
${REQUIRED_LLVM_LIBRARIES}
PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads
${MALLOC_LIBRARIES}
${GLIBC_COMPATIBILITY_LIBRARIES}
${MEMCPY_LIBRARIES}
)

View File

@ -0,0 +1,243 @@
//===-- cc1_main.cpp - Clang CC1 Compiler Frontend ------------------------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
// This is the entry point to the clang -cc1 functionality, which implements the
// core compiler functionality along with a number of additional tools for
// demonstration and testing purposes.
//
//===----------------------------------------------------------------------===//
#include "llvm/Option/Arg.h"
#include "clang/CodeGen/ObjectFilePCHContainerOperations.h"
#include "clang/Config/config.h"
#include "clang/Driver/DriverDiagnostic.h"
#include "clang/Driver/Options.h"
#include "clang/Frontend/CompilerInstance.h"
#include "clang/Frontend/CompilerInvocation.h"
#include "clang/Frontend/FrontendDiagnostic.h"
#include "clang/Frontend/TextDiagnosticBuffer.h"
#include "clang/Frontend/TextDiagnosticPrinter.h"
#include "clang/Frontend/Utils.h"
#include "clang/FrontendTool/Utils.h"
#include "llvm/ADT/Statistic.h"
#include "llvm/Config/llvm-config.h"
#include "llvm/LinkAllPasses.h"
#include "llvm/Option/ArgList.h"
#include "llvm/Option/OptTable.h"
#include "llvm/Support/Compiler.h"
#include "llvm/Support/ErrorHandling.h"
#include "llvm/Support/ManagedStatic.h"
#include "llvm/Support/Signals.h"
#include "llvm/Support/TargetSelect.h"
#include "llvm/Support/Timer.h"
#include "llvm/Support/raw_ostream.h"
#include <cstdio>
#ifdef CLANG_HAVE_RLIMITS
#include <sys/resource.h>
#endif
// have no .a version in packages
#undef LINK_POLLY_INTO_TOOLS
using namespace clang;
using namespace llvm::opt;
//===----------------------------------------------------------------------===//
// Main driver
//===----------------------------------------------------------------------===//
static void LLVMErrorHandler(void *UserData, const std::string &Message,
bool GenCrashDiag) {
DiagnosticsEngine &Diags = *static_cast<DiagnosticsEngine*>(UserData);
Diags.Report(diag::err_fe_error_backend) << Message;
// Run the interrupt handlers to make sure any special cleanups get done, in
// particular that we remove files registered with RemoveFileOnSignal.
llvm::sys::RunInterruptHandlers();
// We cannot recover from llvm errors. When reporting a fatal error, exit
// with status 70 to generate crash diagnostics. For BSD systems this is
// defined as an internal software error. Otherwise, exit with status 1.
exit(GenCrashDiag ? 70 : 1);
}
#ifdef LINK_POLLY_INTO_TOOLS
namespace polly {
void initializePollyPasses(llvm::PassRegistry &Registry);
}
#endif
#ifdef CLANG_HAVE_RLIMITS
// The amount of stack we think is "sufficient". If less than this much is
// available, we may be unable to reach our template instantiation depth
// limit and other similar limits.
// FIXME: Unify this with the stack we request when spawning a thread to build
// a module.
static const int kSufficientStack = 8 << 20;
#if defined(__linux__) && defined(__PIE__)
static size_t getCurrentStackAllocation() {
// If we can't compute the current stack usage, allow for 512K of command
// line arguments and environment.
size_t Usage = 512 * 1024;
if (FILE *StatFile = fopen("/proc/self/stat", "r")) {
// We assume that the stack extends from its current address to the end of
// the environment space. In reality, there is another string literal (the
// program name) after the environment, but this is close enough (we only
// need to be within 100K or so).
unsigned long StackPtr, EnvEnd;
// Disable silly GCC -Wformat warning that complains about length
// modifiers on ignored format specifiers. We want to retain these
// for documentation purposes even though they have no effect.
#if defined(__GNUC__) && !defined(__clang__)
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wformat"
#endif
if (fscanf(StatFile,
"%*d %*s %*c %*d %*d %*d %*d %*d %*u %*lu %*lu %*lu %*lu %*lu "
"%*lu %*ld %*ld %*ld %*ld %*ld %*ld %*llu %*lu %*ld %*lu %*lu "
"%*lu %*lu %lu %*lu %*lu %*lu %*lu %*lu %*llu %*lu %*lu %*d %*d "
"%*u %*u %*llu %*lu %*ld %*lu %*lu %*lu %*lu %*lu %*lu %lu %*d",
&StackPtr, &EnvEnd) == 2) {
#if defined(__GNUC__) && !defined(__clang__)
#pragma GCC diagnostic pop
#endif
Usage = StackPtr < EnvEnd ? EnvEnd - StackPtr : StackPtr - EnvEnd;
}
fclose(StatFile);
}
return Usage;
}
#include <alloca.h>
LLVM_ATTRIBUTE_NOINLINE
static void ensureStackAddressSpace(int ExtraChunks = 0) {
// Linux kernels prior to 4.1 will sometimes locate the heap of a PIE binary
// relatively close to the stack (they are only guaranteed to be 128MiB
// apart). This results in crashes if we happen to heap-allocate more than
// 128MiB before we reach our stack high-water mark.
//
// To avoid these crashes, ensure that we have sufficient virtual memory
// pages allocated before we start running.
size_t Curr = getCurrentStackAllocation();
const int kTargetStack = kSufficientStack - 256 * 1024;
if (Curr < kTargetStack) {
volatile char *volatile Alloc =
static_cast<volatile char *>(alloca(kTargetStack - Curr));
Alloc[0] = 0;
Alloc[kTargetStack - Curr - 1] = 0;
}
}
#else
static void ensureStackAddressSpace() {}
#endif
/// Attempt to ensure that we have at least 8MiB of usable stack space.
static void ensureSufficientStack() {
struct rlimit rlim;
if (getrlimit(RLIMIT_STACK, &rlim) != 0)
return;
// Increase the soft stack limit to our desired level, if necessary and
// possible.
if (rlim.rlim_cur != RLIM_INFINITY && rlim.rlim_cur < kSufficientStack) {
// Try to allocate sufficient stack.
if (rlim.rlim_max == RLIM_INFINITY || rlim.rlim_max >= kSufficientStack)
rlim.rlim_cur = kSufficientStack;
else if (rlim.rlim_cur == rlim.rlim_max)
return;
else
rlim.rlim_cur = rlim.rlim_max;
if (setrlimit(RLIMIT_STACK, &rlim) != 0 ||
rlim.rlim_cur != kSufficientStack)
return;
}
// We should now have a stack of size at least kSufficientStack. Ensure
// that we can actually use that much, if necessary.
ensureStackAddressSpace();
}
#else
static void ensureSufficientStack() {}
#endif
int cc1_main(ArrayRef<const char *> Argv, const char *Argv0, void *MainAddr) {
ensureSufficientStack();
std::unique_ptr<CompilerInstance> Clang(new CompilerInstance());
IntrusiveRefCntPtr<DiagnosticIDs> DiagID(new DiagnosticIDs());
// Register the support for object-file-wrapped Clang modules.
auto PCHOps = Clang->getPCHContainerOperations();
PCHOps->registerWriter(llvm::make_unique<ObjectFilePCHContainerWriter>());
PCHOps->registerReader(llvm::make_unique<ObjectFilePCHContainerReader>());
// Initialize targets first, so that --version shows registered targets.
llvm::InitializeAllTargets();
llvm::InitializeAllTargetMCs();
llvm::InitializeAllAsmPrinters();
llvm::InitializeAllAsmParsers();
#ifdef LINK_POLLY_INTO_TOOLS
llvm::PassRegistry &Registry = *llvm::PassRegistry::getPassRegistry();
polly::initializePollyPasses(Registry);
#endif
// Buffer diagnostics from argument parsing so that we can output them using a
// well formed diagnostic object.
IntrusiveRefCntPtr<DiagnosticOptions> DiagOpts = new DiagnosticOptions();
TextDiagnosticBuffer *DiagsBuffer = new TextDiagnosticBuffer;
DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagsBuffer);
bool Success = CompilerInvocation::CreateFromArgs(
Clang->getInvocation(), Argv.begin(), Argv.end(), Diags);
// Infer the builtin include path if unspecified.
if (Clang->getHeaderSearchOpts().UseBuiltinIncludes &&
Clang->getHeaderSearchOpts().ResourceDir.empty())
Clang->getHeaderSearchOpts().ResourceDir =
CompilerInvocation::GetResourcesPath(Argv0, MainAddr);
// Create the actual diagnostics engine.
Clang->createDiagnostics();
if (!Clang->hasDiagnostics())
return 1;
// Set an error handler, so that any LLVM backend diagnostics go through our
// error handler.
llvm::install_fatal_error_handler(LLVMErrorHandler,
static_cast<void*>(&Clang->getDiagnostics()));
DiagsBuffer->FlushDiagnostics(Clang->getDiagnostics());
if (!Success)
return 1;
// Execute the frontend actions.
Success = ExecuteCompilerInvocation(Clang.get());
// If any timers were active but haven't been destroyed yet, print their
// results now. This happens in -disable-free mode.
llvm::TimerGroup::printAll(llvm::errs());
// Our error handler depends on the Diagnostics object, which we're
// potentially about to delete. Uninstall the handler now so that any
// later errors use the default handling behavior instead.
llvm::remove_fatal_error_handler();
// When running with -disable-free, don't do any destruction or shutdown.
if (Clang->getFrontendOpts().DisableFree) {
BuryPointer(std::move(Clang));
return !Success;
}
return !Success;
}

View File

@ -0,0 +1,555 @@
//===-- cc1as_main.cpp - Clang Assembler ---------------------------------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
// This is the entry point to the clang -cc1as functionality, which implements
// the direct interface to the LLVM MC based assembler.
//
//===----------------------------------------------------------------------===//
#include "clang/Basic/Diagnostic.h"
#include "clang/Basic/DiagnosticOptions.h"
#include "clang/Driver/DriverDiagnostic.h"
#include "clang/Driver/Options.h"
#include "clang/Frontend/FrontendDiagnostic.h"
#include "clang/Frontend/TextDiagnosticPrinter.h"
#include "clang/Frontend/Utils.h"
#include "llvm/ADT/STLExtras.h"
#include "llvm/ADT/StringSwitch.h"
#include "llvm/ADT/Triple.h"
#include "llvm/IR/DataLayout.h"
#include "llvm/MC/MCAsmBackend.h"
#include "llvm/MC/MCAsmInfo.h"
#include "llvm/MC/MCCodeEmitter.h"
#include "llvm/MC/MCContext.h"
#include "llvm/MC/MCInstrInfo.h"
#include "llvm/MC/MCObjectFileInfo.h"
#include "llvm/MC/MCParser/MCAsmParser.h"
#include "llvm/MC/MCParser/MCTargetAsmParser.h"
#include "llvm/MC/MCRegisterInfo.h"
#include "llvm/MC/MCStreamer.h"
#include "llvm/MC/MCSubtargetInfo.h"
#include "llvm/MC/MCTargetOptions.h"
#include "llvm/Option/Arg.h"
#include "llvm/Option/ArgList.h"
#include "llvm/Option/OptTable.h"
#include "llvm/Support/CommandLine.h"
#include "llvm/Support/ErrorHandling.h"
#include "llvm/Support/FileSystem.h"
#include "llvm/Support/FormattedStream.h"
#include "llvm/Support/Host.h"
#include "llvm/Support/MemoryBuffer.h"
#include "llvm/Support/Path.h"
#include "llvm/Support/Signals.h"
#include "llvm/Support/SourceMgr.h"
#include "llvm/Support/TargetRegistry.h"
#include "llvm/Support/TargetSelect.h"
#include "llvm/Support/Timer.h"
#include "llvm/Support/raw_ostream.h"
#include <memory>
#include <system_error>
using namespace clang;
using namespace clang::driver;
using namespace clang::driver::options;
using namespace llvm;
using namespace llvm::opt;
namespace {
/// \brief Helper class for representing a single invocation of the assembler.
struct AssemblerInvocation {
/// @name Target Options
/// @{
/// The name of the target triple to assemble for.
std::string Triple;
/// If given, the name of the target CPU to determine which instructions
/// are legal.
std::string CPU;
/// The list of target specific features to enable or disable -- this should
/// be a list of strings starting with '+' or '-'.
std::vector<std::string> Features;
/// The list of symbol definitions.
std::vector<std::string> SymbolDefs;
/// @}
/// @name Language Options
/// @{
std::vector<std::string> IncludePaths;
unsigned NoInitialTextSection : 1;
unsigned SaveTemporaryLabels : 1;
unsigned GenDwarfForAssembly : 1;
unsigned RelaxELFRelocations : 1;
unsigned DwarfVersion;
std::string DwarfDebugFlags;
std::string DwarfDebugProducer;
std::string DebugCompilationDir;
llvm::DebugCompressionType CompressDebugSections =
llvm::DebugCompressionType::None;
std::string MainFileName;
/// @}
/// @name Frontend Options
/// @{
std::string InputFile;
std::vector<std::string> LLVMArgs;
std::string OutputPath;
enum FileType {
FT_Asm, ///< Assembly (.s) output, transliterate mode.
FT_Null, ///< No output, for timing purposes.
FT_Obj ///< Object file output.
};
FileType OutputType;
unsigned ShowHelp : 1;
unsigned ShowVersion : 1;
/// @}
/// @name Transliterate Options
/// @{
unsigned OutputAsmVariant;
unsigned ShowEncoding : 1;
unsigned ShowInst : 1;
/// @}
/// @name Assembler Options
/// @{
unsigned RelaxAll : 1;
unsigned NoExecStack : 1;
unsigned FatalWarnings : 1;
unsigned IncrementalLinkerCompatible : 1;
/// The name of the relocation model to use.
std::string RelocationModel;
/// @}
public:
AssemblerInvocation() {
Triple = "";
NoInitialTextSection = 0;
InputFile = "-";
OutputPath = "-";
OutputType = FT_Asm;
OutputAsmVariant = 0;
ShowInst = 0;
ShowEncoding = 0;
RelaxAll = 0;
NoExecStack = 0;
FatalWarnings = 0;
IncrementalLinkerCompatible = 0;
DwarfVersion = 0;
}
static bool CreateFromArgs(AssemblerInvocation &Res,
ArrayRef<const char *> Argv,
DiagnosticsEngine &Diags);
};
}
bool AssemblerInvocation::CreateFromArgs(AssemblerInvocation &Opts,
ArrayRef<const char *> Argv,
DiagnosticsEngine &Diags) {
bool Success = true;
// Parse the arguments.
std::unique_ptr<OptTable> OptTbl(createDriverOptTable());
const unsigned IncludedFlagsBitmask = options::CC1AsOption;
unsigned MissingArgIndex, MissingArgCount;
InputArgList Args = OptTbl->ParseArgs(Argv, MissingArgIndex, MissingArgCount,
IncludedFlagsBitmask);
// Check for missing argument error.
if (MissingArgCount) {
Diags.Report(diag::err_drv_missing_argument)
<< Args.getArgString(MissingArgIndex) << MissingArgCount;
Success = false;
}
// Issue errors on unknown arguments.
for (const Arg *A : Args.filtered(OPT_UNKNOWN)) {
auto ArgString = A->getAsString(Args);
std::string Nearest;
if (OptTbl->findNearest(ArgString, Nearest, IncludedFlagsBitmask) > 1)
Diags.Report(diag::err_drv_unknown_argument) << ArgString;
else
Diags.Report(diag::err_drv_unknown_argument_with_suggestion)
<< ArgString << Nearest;
Success = false;
}
// Construct the invocation.
// Target Options
Opts.Triple = llvm::Triple::normalize(Args.getLastArgValue(OPT_triple));
Opts.CPU = Args.getLastArgValue(OPT_target_cpu);
Opts.Features = Args.getAllArgValues(OPT_target_feature);
// Use the default target triple if unspecified.
if (Opts.Triple.empty())
Opts.Triple = llvm::sys::getDefaultTargetTriple();
// Language Options
Opts.IncludePaths = Args.getAllArgValues(OPT_I);
Opts.NoInitialTextSection = Args.hasArg(OPT_n);
Opts.SaveTemporaryLabels = Args.hasArg(OPT_msave_temp_labels);
// Any DebugInfoKind implies GenDwarfForAssembly.
Opts.GenDwarfForAssembly = Args.hasArg(OPT_debug_info_kind_EQ);
if (const Arg *A = Args.getLastArg(OPT_compress_debug_sections,
OPT_compress_debug_sections_EQ)) {
if (A->getOption().getID() == OPT_compress_debug_sections) {
// TODO: be more clever about the compression type auto-detection
Opts.CompressDebugSections = llvm::DebugCompressionType::GNU;
} else {
Opts.CompressDebugSections =
llvm::StringSwitch<llvm::DebugCompressionType>(A->getValue())
.Case("none", llvm::DebugCompressionType::None)
.Case("zlib", llvm::DebugCompressionType::Z)
.Case("zlib-gnu", llvm::DebugCompressionType::GNU)
.Default(llvm::DebugCompressionType::None);
}
}
Opts.RelaxELFRelocations = Args.hasArg(OPT_mrelax_relocations);
Opts.DwarfVersion = getLastArgIntValue(Args, OPT_dwarf_version_EQ, 2, Diags);
Opts.DwarfDebugFlags = Args.getLastArgValue(OPT_dwarf_debug_flags);
Opts.DwarfDebugProducer = Args.getLastArgValue(OPT_dwarf_debug_producer);
Opts.DebugCompilationDir = Args.getLastArgValue(OPT_fdebug_compilation_dir);
Opts.MainFileName = Args.getLastArgValue(OPT_main_file_name);
// Frontend Options
if (Args.hasArg(OPT_INPUT)) {
bool First = true;
for (const Arg *A : Args.filtered(OPT_INPUT)) {
if (First) {
Opts.InputFile = A->getValue();
First = false;
} else {
Diags.Report(diag::err_drv_unknown_argument) << A->getAsString(Args);
Success = false;
}
}
}
Opts.LLVMArgs = Args.getAllArgValues(OPT_mllvm);
Opts.OutputPath = Args.getLastArgValue(OPT_o);
if (Arg *A = Args.getLastArg(OPT_filetype)) {
StringRef Name = A->getValue();
unsigned OutputType = StringSwitch<unsigned>(Name)
.Case("asm", FT_Asm)
.Case("null", FT_Null)
.Case("obj", FT_Obj)
.Default(~0U);
if (OutputType == ~0U) {
Diags.Report(diag::err_drv_invalid_value) << A->getAsString(Args) << Name;
Success = false;
} else
Opts.OutputType = FileType(OutputType);
}
Opts.ShowHelp = Args.hasArg(OPT_help);
Opts.ShowVersion = Args.hasArg(OPT_version);
// Transliterate Options
Opts.OutputAsmVariant =
getLastArgIntValue(Args, OPT_output_asm_variant, 0, Diags);
Opts.ShowEncoding = Args.hasArg(OPT_show_encoding);
Opts.ShowInst = Args.hasArg(OPT_show_inst);
// Assemble Options
Opts.RelaxAll = Args.hasArg(OPT_mrelax_all);
Opts.NoExecStack = Args.hasArg(OPT_mno_exec_stack);
Opts.FatalWarnings = Args.hasArg(OPT_massembler_fatal_warnings);
Opts.RelocationModel = Args.getLastArgValue(OPT_mrelocation_model, "pic");
Opts.IncrementalLinkerCompatible =
Args.hasArg(OPT_mincremental_linker_compatible);
Opts.SymbolDefs = Args.getAllArgValues(OPT_defsym);
return Success;
}
static std::unique_ptr<raw_fd_ostream>
getOutputStream(AssemblerInvocation &Opts, DiagnosticsEngine &Diags,
bool Binary) {
if (Opts.OutputPath.empty())
Opts.OutputPath = "-";
// Make sure that the Out file gets unlinked from the disk if we get a
// SIGINT.
if (Opts.OutputPath != "-")
sys::RemoveFileOnSignal(Opts.OutputPath);
std::error_code EC;
auto Out = llvm::make_unique<raw_fd_ostream>(
Opts.OutputPath, EC, (Binary ? sys::fs::F_None : sys::fs::F_Text));
if (EC) {
Diags.Report(diag::err_fe_unable_to_open_output) << Opts.OutputPath
<< EC.message();
return nullptr;
}
return Out;
}
static bool ExecuteAssembler(AssemblerInvocation &Opts,
DiagnosticsEngine &Diags) {
// Get the target specific parser.
std::string Error;
const Target *TheTarget = TargetRegistry::lookupTarget(Opts.Triple, Error);
if (!TheTarget)
return Diags.Report(diag::err_target_unknown_triple) << Opts.Triple;
ErrorOr<std::unique_ptr<MemoryBuffer>> Buffer =
MemoryBuffer::getFileOrSTDIN(Opts.InputFile);
if (std::error_code EC = Buffer.getError()) {
Error = EC.message();
return Diags.Report(diag::err_fe_error_reading) << Opts.InputFile;
}
SourceMgr SrcMgr;
// Tell SrcMgr about this buffer, which is what the parser will pick up.
SrcMgr.AddNewSourceBuffer(std::move(*Buffer), SMLoc());
// Record the location of the include directories so that the lexer can find
// it later.
SrcMgr.setIncludeDirs(Opts.IncludePaths);
std::unique_ptr<MCRegisterInfo> MRI(TheTarget->createMCRegInfo(Opts.Triple));
assert(MRI && "Unable to create target register info!");
std::unique_ptr<MCAsmInfo> MAI(TheTarget->createMCAsmInfo(*MRI, Opts.Triple));
assert(MAI && "Unable to create target asm info!");
// Ensure MCAsmInfo initialization occurs before any use, otherwise sections
// may be created with a combination of default and explicit settings.
MAI->setCompressDebugSections(Opts.CompressDebugSections);
MAI->setRelaxELFRelocations(Opts.RelaxELFRelocations);
bool IsBinary = Opts.OutputType == AssemblerInvocation::FT_Obj;
std::unique_ptr<raw_fd_ostream> FDOS = getOutputStream(Opts, Diags, IsBinary);
if (!FDOS)
return true;
// FIXME: This is not pretty. MCContext has a ptr to MCObjectFileInfo and
// MCObjectFileInfo needs a MCContext reference in order to initialize itself.
std::unique_ptr<MCObjectFileInfo> MOFI(new MCObjectFileInfo());
MCContext Ctx(MAI.get(), MRI.get(), MOFI.get(), &SrcMgr);
bool PIC = false;
if (Opts.RelocationModel == "static") {
PIC = false;
} else if (Opts.RelocationModel == "pic") {
PIC = true;
} else {
assert(Opts.RelocationModel == "dynamic-no-pic" &&
"Invalid PIC model!");
PIC = false;
}
MOFI->InitMCObjectFileInfo(Triple(Opts.Triple), PIC, Ctx);
if (Opts.SaveTemporaryLabels)
Ctx.setAllowTemporaryLabels(false);
if (Opts.GenDwarfForAssembly)
Ctx.setGenDwarfForAssembly(true);
if (!Opts.DwarfDebugFlags.empty())
Ctx.setDwarfDebugFlags(StringRef(Opts.DwarfDebugFlags));
if (!Opts.DwarfDebugProducer.empty())
Ctx.setDwarfDebugProducer(StringRef(Opts.DwarfDebugProducer));
if (!Opts.DebugCompilationDir.empty())
Ctx.setCompilationDir(Opts.DebugCompilationDir);
if (!Opts.MainFileName.empty())
Ctx.setMainFileName(StringRef(Opts.MainFileName));
Ctx.setDwarfVersion(Opts.DwarfVersion);
// Build up the feature string from the target feature list.
std::string FS;
if (!Opts.Features.empty()) {
FS = Opts.Features[0];
for (unsigned i = 1, e = Opts.Features.size(); i != e; ++i)
FS += "," + Opts.Features[i];
}
std::unique_ptr<MCStreamer> Str;
std::unique_ptr<MCInstrInfo> MCII(TheTarget->createMCInstrInfo());
std::unique_ptr<MCSubtargetInfo> STI(
TheTarget->createMCSubtargetInfo(Opts.Triple, Opts.CPU, FS));
raw_pwrite_stream *Out = FDOS.get();
std::unique_ptr<buffer_ostream> BOS;
// FIXME: There is a bit of code duplication with addPassesToEmitFile.
if (Opts.OutputType == AssemblerInvocation::FT_Asm) {
MCInstPrinter *IP = TheTarget->createMCInstPrinter(
llvm::Triple(Opts.Triple), Opts.OutputAsmVariant, *MAI, *MCII, *MRI);
std::unique_ptr<MCCodeEmitter> CE;
if (Opts.ShowEncoding)
CE.reset(TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx));
MCTargetOptions MCOptions;
std::unique_ptr<MCAsmBackend> MAB(
TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions));
auto FOut = llvm::make_unique<formatted_raw_ostream>(*Out);
Str.reset(TheTarget->createAsmStreamer(
Ctx, std::move(FOut), /*asmverbose*/ true,
/*useDwarfDirectory*/ true, IP, std::move(CE), std::move(MAB),
Opts.ShowInst));
} else if (Opts.OutputType == AssemblerInvocation::FT_Null) {
Str.reset(createNullStreamer(Ctx));
} else {
assert(Opts.OutputType == AssemblerInvocation::FT_Obj &&
"Invalid file type!");
if (!FDOS->supportsSeeking()) {
BOS = make_unique<buffer_ostream>(*FDOS);
Out = BOS.get();
}
std::unique_ptr<MCCodeEmitter> CE(
TheTarget->createMCCodeEmitter(*MCII, *MRI, Ctx));
MCTargetOptions MCOptions;
std::unique_ptr<MCAsmBackend> MAB(
TheTarget->createMCAsmBackend(*STI, *MRI, MCOptions));
Triple T(Opts.Triple);
Str.reset(TheTarget->createMCObjectStreamer(
T, Ctx, std::move(MAB), *Out, std::move(CE), *STI, Opts.RelaxAll,
Opts.IncrementalLinkerCompatible,
/*DWARFMustBeAtTheEnd*/ true));
Str.get()->InitSections(Opts.NoExecStack);
}
// Assembly to object compilation should leverage assembly info.
Str->setUseAssemblerInfoForParsing(true);
bool Failed = false;
std::unique_ptr<MCAsmParser> Parser(
createMCAsmParser(SrcMgr, Ctx, *Str.get(), *MAI));
// FIXME: init MCTargetOptions from sanitizer flags here.
MCTargetOptions Options;
std::unique_ptr<MCTargetAsmParser> TAP(
TheTarget->createMCAsmParser(*STI, *Parser, *MCII, Options));
if (!TAP)
Failed = Diags.Report(diag::err_target_unknown_triple) << Opts.Triple;
// Set values for symbols, if any.
for (auto &S : Opts.SymbolDefs) {
auto Pair = StringRef(S).split('=');
auto Sym = Pair.first;
auto Val = Pair.second;
int64_t Value;
// We have already error checked this in the driver.
Val.getAsInteger(0, Value);
Ctx.setSymbolValue(Parser->getStreamer(), Sym, Value);
}
if (!Failed) {
Parser->setTargetParser(*TAP.get());
Failed = Parser->Run(Opts.NoInitialTextSection);
}
// Close Streamer first.
// It might have a reference to the output stream.
Str.reset();
// Close the output stream early.
BOS.reset();
FDOS.reset();
// Delete output file if there were errors.
if (Failed && Opts.OutputPath != "-")
sys::fs::remove(Opts.OutputPath);
return Failed;
}
static void LLVMErrorHandler(void *UserData, const std::string &Message,
bool /*GenCrashDiag*/) {
DiagnosticsEngine &Diags = *static_cast<DiagnosticsEngine*>(UserData);
Diags.Report(diag::err_fe_error_backend) << Message;
// We cannot recover from llvm errors.
exit(1);
}
int cc1as_main(ArrayRef<const char *> Argv, const char */*Argv0*/, void */*MainAddr*/) {
// Initialize targets and assembly printers/parsers.
InitializeAllTargetInfos();
InitializeAllTargetMCs();
InitializeAllAsmParsers();
// Construct our diagnostic client.
IntrusiveRefCntPtr<DiagnosticOptions> DiagOpts = new DiagnosticOptions();
TextDiagnosticPrinter *DiagClient
= new TextDiagnosticPrinter(errs(), &*DiagOpts);
DiagClient->setPrefix("clang -cc1as");
IntrusiveRefCntPtr<DiagnosticIDs> DiagID(new DiagnosticIDs());
DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient);
// Set an error handler, so that any LLVM backend diagnostics go through our
// error handler.
ScopedFatalErrorHandler FatalErrorHandler
(LLVMErrorHandler, static_cast<void*>(&Diags));
// Parse the arguments.
AssemblerInvocation Asm;
if (!AssemblerInvocation::CreateFromArgs(Asm, Argv, Diags))
return 1;
if (Asm.ShowHelp) {
std::unique_ptr<OptTable> Opts(driver::createDriverOptTable());
Opts->PrintHelp(llvm::outs(), "clang -cc1as", "Clang Integrated Assembler",
/*Include=*/driver::options::CC1AsOption, /*Exclude=*/0,
/*ShowAllAliases=*/false);
return 0;
}
// Honor -version.
//
// FIXME: Use a better -version message?
if (Asm.ShowVersion) {
llvm::cl::PrintVersionMessage();
return 0;
}
// Honor -mllvm.
//
// FIXME: Remove this, one day.
if (!Asm.LLVMArgs.empty()) {
unsigned NumArgs = Asm.LLVMArgs.size();
auto Args = llvm::make_unique<const char*[]>(NumArgs + 2);
Args[0] = "clang (LLVM option parsing)";
for (unsigned i = 0; i != NumArgs; ++i)
Args[i + 1] = Asm.LLVMArgs[i].c_str();
Args[NumArgs + 1] = nullptr;
llvm::cl::ParseCommandLineOptions(NumArgs + 1, Args.get());
}
// Execute the invocation, unless there were parsing errors.
bool Failed = Diags.hasErrorOccurred() || ExecuteAssembler(Asm, Diags);
// If any timers were active but haven't been destroyed yet, print their
// results now.
TimerGroup::printAll(errs());
return !!Failed;
}

View File

@ -0,0 +1,512 @@
//===-- driver.cpp - Clang GCC-Compatible Driver --------------------------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
// This is the entry point to the clang driver; it is a thin wrapper
// for functionality in the Driver clang library.
//
//===----------------------------------------------------------------------===//
#include "clang/Driver/Driver.h"
#include "clang/Basic/DiagnosticOptions.h"
#include "clang/Driver/Compilation.h"
#include "clang/Driver/DriverDiagnostic.h"
#include "clang/Driver/Options.h"
#include "clang/Driver/ToolChain.h"
#include "clang/Frontend/ChainedDiagnosticConsumer.h"
#include "clang/Frontend/CompilerInvocation.h"
#include "clang/Frontend/SerializedDiagnosticPrinter.h"
#include "clang/Frontend/TextDiagnosticPrinter.h"
#include "clang/Frontend/Utils.h"
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ADT/SmallString.h"
#include "llvm/ADT/SmallVector.h"
#include "llvm/Option/ArgList.h"
#include "llvm/Option/OptTable.h"
#include "llvm/Option/Option.h"
#include "llvm/Support/CommandLine.h"
#include "llvm/Support/ErrorHandling.h"
#include "llvm/Support/FileSystem.h"
#include "llvm/Support/Host.h"
#include "llvm/Support/InitLLVM.h"
#include "llvm/Support/Path.h"
#include "llvm/Support/Process.h"
#include "llvm/Support/Program.h"
#include "llvm/Support/Regex.h"
#include "llvm/Support/Signals.h"
#include "llvm/Support/StringSaver.h"
#include "llvm/Support/TargetSelect.h"
#include "llvm/Support/Timer.h"
#include "llvm/Support/raw_ostream.h"
#include <memory>
#include <set>
#include <system_error>
using namespace clang;
using namespace clang::driver;
using namespace llvm::opt;
std::string GetExecutablePath(const char *Argv0, bool CanonicalPrefixes) {
if (!CanonicalPrefixes) {
SmallString<128> ExecutablePath(Argv0);
// Do a PATH lookup if Argv0 isn't a valid path.
if (!llvm::sys::fs::exists(ExecutablePath))
if (llvm::ErrorOr<std::string> P =
llvm::sys::findProgramByName(ExecutablePath))
ExecutablePath = *P;
return ExecutablePath.str();
}
// This just needs to be some symbol in the binary; C++ doesn't
// allow taking the address of ::main however.
void *P = (void*) (intptr_t) GetExecutablePath;
return llvm::sys::fs::getMainExecutable(Argv0, P);
}
static const char *GetStableCStr(std::set<std::string> &SavedStrings,
StringRef S) {
return SavedStrings.insert(S).first->c_str();
}
/// ApplyQAOverride - Apply a list of edits to the input argument lists.
///
/// The input string is a space separate list of edits to perform,
/// they are applied in order to the input argument lists. Edits
/// should be one of the following forms:
///
/// '#': Silence information about the changes to the command line arguments.
///
/// '^': Add FOO as a new argument at the beginning of the command line.
///
/// '+': Add FOO as a new argument at the end of the command line.
///
/// 's/XXX/YYY/': Substitute the regular expression XXX with YYY in the command
/// line.
///
/// 'xOPTION': Removes all instances of the literal argument OPTION.
///
/// 'XOPTION': Removes all instances of the literal argument OPTION,
/// and the following argument.
///
/// 'Ox': Removes all flags matching 'O' or 'O[sz0-9]' and adds 'Ox'
/// at the end of the command line.
///
/// \param OS - The stream to write edit information to.
/// \param Args - The vector of command line arguments.
/// \param Edit - The override command to perform.
/// \param SavedStrings - Set to use for storing string representations.
static void ApplyOneQAOverride(raw_ostream &OS,
SmallVectorImpl<const char*> &Args,
StringRef Edit,
std::set<std::string> &SavedStrings) {
// This does not need to be efficient.
if (Edit[0] == '^') {
const char *Str =
GetStableCStr(SavedStrings, Edit.substr(1));
OS << "### Adding argument " << Str << " at beginning\n";
Args.insert(Args.begin() + 1, Str);
} else if (Edit[0] == '+') {
const char *Str =
GetStableCStr(SavedStrings, Edit.substr(1));
OS << "### Adding argument " << Str << " at end\n";
Args.push_back(Str);
} else if (Edit[0] == 's' && Edit[1] == '/' && Edit.endswith("/") &&
Edit.slice(2, Edit.size()-1).find('/') != StringRef::npos) {
StringRef MatchPattern = Edit.substr(2).split('/').first;
StringRef ReplPattern = Edit.substr(2).split('/').second;
ReplPattern = ReplPattern.slice(0, ReplPattern.size()-1);
for (unsigned i = 1, e = Args.size(); i != e; ++i) {
// Ignore end-of-line response file markers
if (Args[i] == nullptr)
continue;
std::string Repl = llvm::Regex(MatchPattern).sub(ReplPattern, Args[i]);
if (Repl != Args[i]) {
OS << "### Replacing '" << Args[i] << "' with '" << Repl << "'\n";
Args[i] = GetStableCStr(SavedStrings, Repl);
}
}
} else if (Edit[0] == 'x' || Edit[0] == 'X') {
auto Option = Edit.substr(1);
for (unsigned i = 1; i < Args.size();) {
if (Option == Args[i]) {
OS << "### Deleting argument " << Args[i] << '\n';
Args.erase(Args.begin() + i);
if (Edit[0] == 'X') {
if (i < Args.size()) {
OS << "### Deleting argument " << Args[i] << '\n';
Args.erase(Args.begin() + i);
} else
OS << "### Invalid X edit, end of command line!\n";
}
} else
++i;
}
} else if (Edit[0] == 'O') {
for (unsigned i = 1; i < Args.size();) {
const char *A = Args[i];
// Ignore end-of-line response file markers
if (A == nullptr)
continue;
if (A[0] == '-' && A[1] == 'O' &&
(A[2] == '\0' ||
(A[3] == '\0' && (A[2] == 's' || A[2] == 'z' ||
('0' <= A[2] && A[2] <= '9'))))) {
OS << "### Deleting argument " << Args[i] << '\n';
Args.erase(Args.begin() + i);
} else
++i;
}
OS << "### Adding argument " << Edit << " at end\n";
Args.push_back(GetStableCStr(SavedStrings, '-' + Edit.str()));
} else {
OS << "### Unrecognized edit: " << Edit << "\n";
}
}
/// ApplyQAOverride - Apply a comma separate list of edits to the
/// input argument lists. See ApplyOneQAOverride.
static void ApplyQAOverride(SmallVectorImpl<const char*> &Args,
const char *OverrideStr,
std::set<std::string> &SavedStrings) {
raw_ostream *OS = &llvm::errs();
if (OverrideStr[0] == '#') {
++OverrideStr;
OS = &llvm::nulls();
}
*OS << "### CCC_OVERRIDE_OPTIONS: " << OverrideStr << "\n";
// This does not need to be efficient.
const char *S = OverrideStr;
while (*S) {
const char *End = ::strchr(S, ' ');
if (!End)
End = S + strlen(S);
if (End != S)
ApplyOneQAOverride(*OS, Args, std::string(S, End), SavedStrings);
S = End;
if (*S != '\0')
++S;
}
}
extern int cc1_main(ArrayRef<const char *> Argv, const char *Argv0,
void *MainAddr);
extern int cc1as_main(ArrayRef<const char *> Argv, const char *Argv0,
void *MainAddr);
extern int cc1gen_reproducer_main(ArrayRef<const char *> Argv,
const char *Argv0, void *MainAddr);
static void insertTargetAndModeArgs(const ParsedClangName &NameParts,
SmallVectorImpl<const char *> &ArgVector,
std::set<std::string> &SavedStrings) {
// Put target and mode arguments at the start of argument list so that
// arguments specified in command line could override them. Avoid putting
// them at index 0, as an option like '-cc1' must remain the first.
int InsertionPoint = 0;
if (ArgVector.size() > 0)
++InsertionPoint;
if (NameParts.DriverMode) {
// Add the mode flag to the arguments.
ArgVector.insert(ArgVector.begin() + InsertionPoint,
GetStableCStr(SavedStrings, NameParts.DriverMode));
}
if (NameParts.TargetIsValid) {
const char *arr[] = {"-target", GetStableCStr(SavedStrings,
NameParts.TargetPrefix)};
ArgVector.insert(ArgVector.begin() + InsertionPoint,
std::begin(arr), std::end(arr));
}
}
static void getCLEnvVarOptions(std::string &EnvValue, llvm::StringSaver &Saver,
SmallVectorImpl<const char *> &Opts) {
llvm::cl::TokenizeWindowsCommandLine(EnvValue, Saver, Opts);
// The first instance of '#' should be replaced with '=' in each option.
for (const char *Opt : Opts)
if (char *NumberSignPtr = const_cast<char *>(::strchr(Opt, '#')))
*NumberSignPtr = '=';
}
static void SetBackdoorDriverOutputsFromEnvVars(Driver &TheDriver) {
// Handle CC_PRINT_OPTIONS and CC_PRINT_OPTIONS_FILE.
TheDriver.CCPrintOptions = !!::getenv("CC_PRINT_OPTIONS");
if (TheDriver.CCPrintOptions)
TheDriver.CCPrintOptionsFilename = ::getenv("CC_PRINT_OPTIONS_FILE");
// Handle CC_PRINT_HEADERS and CC_PRINT_HEADERS_FILE.
TheDriver.CCPrintHeaders = !!::getenv("CC_PRINT_HEADERS");
if (TheDriver.CCPrintHeaders)
TheDriver.CCPrintHeadersFilename = ::getenv("CC_PRINT_HEADERS_FILE");
// Handle CC_LOG_DIAGNOSTICS and CC_LOG_DIAGNOSTICS_FILE.
TheDriver.CCLogDiagnostics = !!::getenv("CC_LOG_DIAGNOSTICS");
if (TheDriver.CCLogDiagnostics)
TheDriver.CCLogDiagnosticsFilename = ::getenv("CC_LOG_DIAGNOSTICS_FILE");
}
static void FixupDiagPrefixExeName(TextDiagnosticPrinter *DiagClient,
const std::string &Path) {
// If the clang binary happens to be named cl.exe for compatibility reasons,
// use clang-cl.exe as the prefix to avoid confusion between clang and MSVC.
StringRef ExeBasename(llvm::sys::path::filename(Path));
if (ExeBasename.equals_lower("cl.exe"))
ExeBasename = "clang-cl.exe";
DiagClient->setPrefix(ExeBasename);
}
// This lets us create the DiagnosticsEngine with a properly-filled-out
// DiagnosticOptions instance.
static DiagnosticOptions *
CreateAndPopulateDiagOpts(ArrayRef<const char *> argv) {
auto *DiagOpts = new DiagnosticOptions;
std::unique_ptr<OptTable> Opts(createDriverOptTable());
unsigned MissingArgIndex, MissingArgCount;
InputArgList Args =
Opts->ParseArgs(argv.slice(1), MissingArgIndex, MissingArgCount);
// We ignore MissingArgCount and the return value of ParseDiagnosticArgs.
// Any errors that would be diagnosed here will also be diagnosed later,
// when the DiagnosticsEngine actually exists.
(void)ParseDiagnosticArgs(*DiagOpts, Args);
return DiagOpts;
}
static void SetInstallDir(SmallVectorImpl<const char *> &argv,
Driver &TheDriver, bool CanonicalPrefixes) {
// Attempt to find the original path used to invoke the driver, to determine
// the installed path. We do this manually, because we want to support that
// path being a symlink.
SmallString<128> InstalledPath(argv[0]);
// Do a PATH lookup, if there are no directory components.
if (llvm::sys::path::filename(InstalledPath) == InstalledPath)
if (llvm::ErrorOr<std::string> Tmp = llvm::sys::findProgramByName(
llvm::sys::path::filename(InstalledPath.str())))
InstalledPath = *Tmp;
// FIXME: We don't actually canonicalize this, we just make it absolute.
if (CanonicalPrefixes)
llvm::sys::fs::make_absolute(InstalledPath);
StringRef InstalledPathParent(llvm::sys::path::parent_path(InstalledPath));
if (llvm::sys::fs::exists(InstalledPathParent))
TheDriver.setInstalledDir(InstalledPathParent);
}
static int ExecuteCC1Tool(ArrayRef<const char *> argv, StringRef Tool) {
void *GetExecutablePathVP = (void *)(intptr_t) GetExecutablePath;
if (Tool == "")
return cc1_main(argv.slice(2), argv[0], GetExecutablePathVP);
if (Tool == "as")
return cc1as_main(argv.slice(2), argv[0], GetExecutablePathVP);
// Reject unknown tools.
llvm::errs() << "error: unknown integrated tool '" << Tool << "'. "
<< "Valid tools include '-cc1' and '-cc1as'.\n";
return 1;
}
int mainEntryClickHouseClang(int argc_, char **argv_) {
llvm::InitLLVM X(argc_, argv_);
SmallVector<const char *, 256> argv(argv_, argv_ + argc_);
if (llvm::sys::Process::FixupStandardFileDescriptors())
return 1;
llvm::InitializeAllTargets();
auto TargetAndMode = ToolChain::getTargetAndModeFromProgramName(argv[0]);
llvm::BumpPtrAllocator A;
llvm::StringSaver Saver(A);
// Parse response files using the GNU syntax, unless we're in CL mode. There
// are two ways to put clang in CL compatibility mode: argv[0] is either
// clang-cl or cl, or --driver-mode=cl is on the command line. The normal
// command line parsing can't happen until after response file parsing, so we
// have to manually search for a --driver-mode=cl argument the hard way.
// Finally, our -cc1 tools don't care which tokenization mode we use because
// response files written by clang will tokenize the same way in either mode.
bool ClangCLMode = false;
if (StringRef(TargetAndMode.DriverMode).equals("--driver-mode=cl") ||
std::find_if(argv.begin(), argv.end(), [](const char *F) {
return F && strcmp(F, "--driver-mode=cl") == 0;
}) != argv.end()) {
ClangCLMode = true;
}
enum { Default, POSIX, Windows } RSPQuoting = Default;
for (const char *F : argv) {
if (strcmp(F, "--rsp-quoting=posix") == 0)
RSPQuoting = POSIX;
else if (strcmp(F, "--rsp-quoting=windows") == 0)
RSPQuoting = Windows;
}
// Determines whether we want nullptr markers in argv to indicate response
// files end-of-lines. We only use this for the /LINK driver argument with
// clang-cl.exe on Windows.
bool MarkEOLs = ClangCLMode;
llvm::cl::TokenizerCallback Tokenizer;
if (RSPQuoting == Windows || (RSPQuoting == Default && ClangCLMode))
Tokenizer = &llvm::cl::TokenizeWindowsCommandLine;
else
Tokenizer = &llvm::cl::TokenizeGNUCommandLine;
if (MarkEOLs && argv.size() > 1 && StringRef(argv[1]).startswith("-cc1"))
MarkEOLs = false;
llvm::cl::ExpandResponseFiles(Saver, Tokenizer, argv, MarkEOLs);
// Handle -cc1 integrated tools, even if -cc1 was expanded from a response
// file.
auto FirstArg = std::find_if(argv.begin() + 1, argv.end(),
[](const char *A) { return A != nullptr; });
if (FirstArg != argv.end() && StringRef(*FirstArg).startswith("-cc1")) {
// If -cc1 came from a response file, remove the EOL sentinels.
if (MarkEOLs) {
auto newEnd = std::remove(argv.begin(), argv.end(), nullptr);
argv.resize(newEnd - argv.begin());
}
return ExecuteCC1Tool(argv, argv[1] + 4);
}
bool CanonicalPrefixes = true;
for (int i = 1, size = argv.size(); i < size; ++i) {
// Skip end-of-line response file markers
if (argv[i] == nullptr)
continue;
if (StringRef(argv[i]) == "-no-canonical-prefixes") {
CanonicalPrefixes = false;
break;
}
}
// Handle CL and _CL_ which permits additional command line options to be
// prepended or appended.
if (ClangCLMode) {
// Arguments in "CL" are prepended.
llvm::Optional<std::string> OptCL = llvm::sys::Process::GetEnv("CL");
if (OptCL.hasValue()) {
SmallVector<const char *, 8> PrependedOpts;
getCLEnvVarOptions(OptCL.getValue(), Saver, PrependedOpts);
// Insert right after the program name to prepend to the argument list.
argv.insert(argv.begin() + 1, PrependedOpts.begin(), PrependedOpts.end());
}
// Arguments in "_CL_" are appended.
llvm::Optional<std::string> Opt_CL_ = llvm::sys::Process::GetEnv("_CL_");
if (Opt_CL_.hasValue()) {
SmallVector<const char *, 8> AppendedOpts;
getCLEnvVarOptions(Opt_CL_.getValue(), Saver, AppendedOpts);
// Insert at the end of the argument list to append.
argv.append(AppendedOpts.begin(), AppendedOpts.end());
}
}
std::set<std::string> SavedStrings;
// Handle CCC_OVERRIDE_OPTIONS, used for editing a command line behind the
// scenes.
if (const char *OverrideStr = ::getenv("CCC_OVERRIDE_OPTIONS")) {
// FIXME: Driver shouldn't take extra initial argument.
ApplyQAOverride(argv, OverrideStr, SavedStrings);
}
std::string Path = GetExecutablePath(argv[0], CanonicalPrefixes);
IntrusiveRefCntPtr<DiagnosticOptions> DiagOpts =
CreateAndPopulateDiagOpts(argv);
TextDiagnosticPrinter *DiagClient
= new TextDiagnosticPrinter(llvm::errs(), &*DiagOpts);
FixupDiagPrefixExeName(DiagClient, Path);
IntrusiveRefCntPtr<DiagnosticIDs> DiagID(new DiagnosticIDs());
DiagnosticsEngine Diags(DiagID, &*DiagOpts, DiagClient);
if (!DiagOpts->DiagnosticSerializationFile.empty()) {
auto SerializedConsumer =
clang::serialized_diags::create(DiagOpts->DiagnosticSerializationFile,
&*DiagOpts, /*MergeChildRecords=*/true);
Diags.setClient(new ChainedDiagnosticConsumer(
Diags.takeClient(), std::move(SerializedConsumer)));
}
ProcessWarningOptions(Diags, *DiagOpts, /*ReportDiags=*/false);
Driver TheDriver(Path, llvm::sys::getDefaultTargetTriple(), Diags);
SetInstallDir(argv, TheDriver, CanonicalPrefixes);
TheDriver.setTargetAndMode(TargetAndMode);
insertTargetAndModeArgs(TargetAndMode, argv, SavedStrings);
SetBackdoorDriverOutputsFromEnvVars(TheDriver);
std::unique_ptr<Compilation> C(TheDriver.BuildCompilation(argv));
int Res = 1;
if (C && !C->containsError()) {
SmallVector<std::pair<int, const Command *>, 4> FailingCommands;
Res = TheDriver.ExecuteCompilation(*C, FailingCommands);
// Force a crash to test the diagnostics.
if (TheDriver.GenReproducer) {
Diags.Report(diag::err_drv_force_crash)
<< !::getenv("FORCE_CLANG_DIAGNOSTICS_CRASH");
// Pretend that every command failed.
FailingCommands.clear();
for (const auto &J : C->getJobs())
if (const Command *C = dyn_cast<Command>(&J))
FailingCommands.push_back(std::make_pair(-1, C));
}
for (const auto &P : FailingCommands) {
int CommandRes = P.first;
const Command *FailingCommand = P.second;
if (!Res)
Res = CommandRes;
// If result status is < 0, then the driver command signalled an error.
// If result status is 70, then the driver command reported a fatal error.
// On Windows, abort will return an exit code of 3. In these cases,
// generate additional diagnostic information if possible.
bool DiagnoseCrash = CommandRes < 0 || CommandRes == 70;
#ifdef _WIN32
DiagnoseCrash |= CommandRes == 3;
#endif
if (DiagnoseCrash) {
TheDriver.generateCompilationDiagnostics(*C, *FailingCommand);
break;
}
}
}
Diags.getClient()->finish();
// If any timers were active but haven't been destroyed yet, print their
// results now. This happens in -disable-free mode.
llvm::TimerGroup::printAll(llvm::errs());
#ifdef _WIN32
// Exit status should not be negative on Win32, unless abnormal termination.
// Once abnormal termiation was caught, negative status should not be
// propagated.
if (Res < 0)
Res = 1;
#endif
// If we have multiple failing commands, we return the result of the first
// failing command.
return Res;
}

View File

@ -0,0 +1,10 @@
#include "lld/Common/Driver.h"
#include "llvm/Support/InitLLVM.h"
#include <vector>
int mainEntryClickHouseLLD(int argc, char ** argv)
{
llvm::InitLLVM X(argc, argv);
std::vector<const char *> args(argv, argv + argc);
return !lld::elf::link(args, false);
}

View File

@ -19,8 +19,8 @@ if (CLICKHOUSE_SPLIT_BINARY)
install (TARGETS clickhouse-server ${CLICKHOUSE_ALL_TARGETS} RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
endif ()
if (OS_LINUX)
set (GLIBC_MAX_REQUIRED 2.4)
if (OS_LINUX AND MAKE_STATIC_LIBRARIES)
set (GLIBC_MAX_REQUIRED 2.4 CACHE INTERNAL "")
add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'")
endif ()

View File

@ -42,6 +42,11 @@
#include <Common/StatusFile.h>
#include "TCPHandlerFactory.h"
#if defined(__linux__)
#include <Common/hasLinuxCapability.h>
#include <sys/mman.h>
#endif
#if USE_POCO_NETSSL
#include <Poco/Net/Context.h>
#include <Poco/Net/SecureServerSocket.h>
@ -125,6 +130,32 @@ int Server::main(const std::vector<std::string> & /*args*/)
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
}
const auto memory_amount = getMemoryAmount();
#if defined(__linux__)
/// After full config loaded
{
if (config().getBool("mlock_executable", false))
{
if (hasLinuxCapability(CAP_IPC_LOCK))
{
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlockall(MCL_CURRENT))
LOG_WARNING(log, "Failed mlockall: " + errnoToString());
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
}
else
{
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'."
" Note that it will not work on 'nosuid' mounted filesystems.");
}
}
}
#endif
std::string path = getCanonicalPath(config().getString("path"));
std::string default_database = config().getString("default_database", "default");
@ -599,7 +630,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
std::stringstream message;
message << "Available RAM = " << formatReadableSizeWithBinarySuffix(getMemoryAmount()) << ";"
message << "Available RAM = " << formatReadableSizeWithBinarySuffix(memory_amount) << ";"
<< " physical cores = " << getNumberOfPhysicalCPUCores() << ";"
// on ARM processors it can show only enabled at current moment cores
<< " threads = " << std::thread::hardware_concurrency() << ".";

View File

@ -145,6 +145,12 @@
-->
<!-- <umask>022</umask> -->
<!-- Perform mlockall after startup to lower first queries latency
and to prevent clickhouse executable from being paged out under high IO load.
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
-->
<mlock_executable>false</mlock_executable>
<!-- Configuration of clusters that could be used in Distributed tables.
https://clickhouse.yandex/docs/en/table_engines/distributed/
-->

View File

@ -1,46 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <unordered_map>
#include <vector>
namespace DB
{
class WriteBuffer;
struct CollectAliases;
struct CollectTables;
/** For every ARRAY JOIN, collect a map:
* result alias -> source
*
* There could be several variants:
*
* SELECT elem FROM t ARRAY JOIN array AS elem elem -> array
* SELECT n.elem FROM t ARRAY JOIN nested AS n n -> nested
* SELECT array FROM t ARRAY JOIN array array -> array
* SELECT nested.elem FROM t ARRAY JOIN nested nested -> nested
* SELECT elem FROM t ARRAY JOIN [1, 2, 3] AS elem elem -> [1, 2, 3]
*
* Does not analyze arrayJoin functions.
*/
struct AnalyzeArrayJoins
{
void process(const ASTPtr & ast);
struct SourceInfo
{
String column_name;
ASTPtr node;
};
using ResultToSource = std::unordered_map<String, SourceInfo>;
using ArrayJoins = std::vector<ResultToSource>;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,455 +0,0 @@
#include <vector>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/CollectAliases.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Poco/String.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int AMBIGUOUS_TABLE_NAME;
extern const int AMBIGUOUS_COLUMN_NAME;
extern const int UNKNOWN_TABLE;
extern const int THERE_IS_NO_COLUMN;
}
namespace
{
/// Find by fully qualified name, like db.table.column
const CollectTables::TableInfo * findTableByDatabaseAndTableName(
const CollectTables & tables, const String & database_name, const String & table_name)
{
for (const auto & table : tables.tables)
if (table.database_name == database_name && table.table_name == table_name)
return &table;
return nullptr;
}
/** Find by single-qualified name, like table.column or alias.column.
*
* There are primary matches:
* when name is alias like
* SELECT name.column FROM (SELECT 1) AS name
* or name is table in current database like
* SELECT name.column FROM name
*
* And secondary matches:
* when name is name of table in explicitly specified database like
* SELECT name.column FROM db.name
*
* If there is only one primary match - return it.
* If there is many primary matches - ambiguity.
* If there is no primary matches and only one secondary match - return it.
* If there is no primary matches and many secondary matches - ambiguity.
* If there is no any matches - not found.
*/
const CollectTables::TableInfo * findTableByNameOrAlias(
const CollectTables & tables, const String & name)
{
const CollectTables::TableInfo * primary_match = nullptr;
const CollectTables::TableInfo * secondary_match = nullptr;
for (const auto & table : tables.tables)
{
if (table.alias == name
|| (table.database_name.empty() && table.table_name == name))
{
if (primary_match)
throw Exception("Table name " + backQuoteIfNeed(name) + " is ambiguous", ErrorCodes::AMBIGUOUS_TABLE_NAME);
primary_match = &table;
}
else if (!primary_match && table.table_name == name)
{
if (secondary_match)
throw Exception("Table name " + backQuoteIfNeed(name) + " is ambiguous", ErrorCodes::AMBIGUOUS_TABLE_NAME);
secondary_match = &table;
}
}
if (primary_match)
return primary_match;
if (secondary_match)
return secondary_match;
return nullptr;
}
/** Find table in case when its name is not specified. Like just
* SELECT column FROM t1, t2
* Select a table, where specified column exists.
* If more than one such table - ambiguity.
*/
const CollectTables::TableInfo * findTableWithUnqualifiedName(const CollectTables & tables, const String & column_name)
{
const CollectTables::TableInfo * res = nullptr;
for (const auto & table : tables.tables)
{
if (table.structure_of_subquery)
{
if (table.structure_of_subquery.has(column_name))
{
if (res)
throw Exception("Ambiguous column name " + backQuoteIfNeed(column_name), ErrorCodes::AMBIGUOUS_COLUMN_NAME);
res = &table;
break;
}
}
else if (table.storage)
{
if (table.storage->hasColumn(column_name))
{
if (res)
throw Exception("Ambiguous column name " + backQuoteIfNeed(column_name), ErrorCodes::AMBIGUOUS_COLUMN_NAME);
res = &table;
}
}
else
throw Exception("Logical error: no storage and no structure of subquery is specified for table", ErrorCodes::LOGICAL_ERROR);
}
return res;
}
/// Create maximum-qualified identifier for column in table.
ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectTables::TableInfo & table)
{
ASTPtr database_name_identifier_node;
if (!table.database_name.empty())
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name);
ASTPtr table_name_identifier_node;
String table_name_or_alias;
if (!table.table_name.empty())
table_name_or_alias = table.table_name;
else if (table.database_name.empty() && !table.alias.empty())
table_name_or_alias = table.alias;
if (!table_name_or_alias.empty())
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias);
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column);
String compound_name;
if (database_name_identifier_node)
compound_name += table.database_name + ".";
if (table_name_identifier_node)
compound_name += table_name_or_alias + ".";
compound_name += column;
auto elem = std::make_shared<ASTIdentifier>(compound_name);
if (database_name_identifier_node)
elem->children.emplace_back(std::move(database_name_identifier_node));
if (table_name_identifier_node)
elem->children.emplace_back(std::move(table_name_identifier_node));
if (!elem->children.empty())
elem->children.emplace_back(std::move(column_identifier_node));
return elem;
}
void createASTsForAllColumnsInTable(const CollectTables::TableInfo & table, ASTs & res)
{
if (table.storage)
for (const auto & name : table.storage->getColumns().getNamesOfPhysical())
res.emplace_back(createASTIdentifierForColumnInTable(name, table));
else
for (size_t i = 0, size = table.structure_of_subquery.columns(); i < size; ++i)
res.emplace_back(createASTIdentifierForColumnInTable(table.structure_of_subquery.getByPosition(i).name, table));
}
ASTs expandUnqualifiedAsterisk(const CollectTables & tables)
{
ASTs res;
for (const auto & table : tables.tables)
createASTsForAllColumnsInTable(table, res);
return res;
}
ASTs expandQualifiedAsterisk(
const IAST & ast, const CollectTables & tables)
{
if (ast.children.size() != 1)
throw Exception("Logical error: AST node for qualified asterisk has number of children not equal to one", ErrorCodes::LOGICAL_ERROR);
const ASTIdentifier & qualifier = static_cast<const ASTIdentifier &>(*ast.children[0]);
const CollectTables::TableInfo * table = nullptr;
if (qualifier.children.empty())
table = findTableByNameOrAlias(tables, qualifier.name);
else if (qualifier.children.size() == 2)
table = findTableByDatabaseAndTableName(tables,
static_cast<const ASTIdentifier &>(*qualifier.children[0]).name,
static_cast<const ASTIdentifier &>(*qualifier.children[1]).name);
else
throw Exception("Unsupported number of components in asterisk qualifier", ErrorCodes::NOT_IMPLEMENTED);
/// TODO Implement for case table.nested.* and database.table.nested.*
if (!table)
throw Exception("There is no table " + qualifier.name + " in query", ErrorCodes::UNKNOWN_TABLE);
ASTs res;
createASTsForAllColumnsInTable(*table, res);
return res;
}
void processIdentifier(
const ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables)
{
const ASTIdentifier & identifier = static_cast<const ASTIdentifier &>(*ast);
if (aliases.aliases.count(identifier.name))
return;
if (columns.count(identifier.name))
return;
const CollectTables::TableInfo * table = nullptr;
String column_name;
if (identifier.children.empty())
{
/** Lambda parameters are not columns from table. Just skip them.
* This step requires AnalyzeLambdas to be done on AST.
*/
if (startsWith(identifier.name, "_lambda"))
return;
table = findTableWithUnqualifiedName(tables, identifier.name);
if (table)
column_name = identifier.name;
}
else if (identifier.children.size() == 2)
{
const String & first = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
const String & second = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
/// table.column
table = findTableByNameOrAlias(tables, first);
if (table)
{
column_name = second;
}
else
{
/// column.nested
table = findTableWithUnqualifiedName(tables, identifier.name);
if (table)
column_name = identifier.name;
}
}
else if (identifier.children.size() == 3)
{
const String & first = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
const String & second = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
const String & third = static_cast<const ASTIdentifier &>(*identifier.children[2]).name;
/// database.table.column
table = findTableByDatabaseAndTableName(tables, first, second);
if (table)
{
column_name = third;
}
else
{
/// table.column.nested
table = findTableByNameOrAlias(tables, first);
if (table)
{
column_name = second + "." + third;
}
else
{
/// column.nested.nested
table = findTableWithUnqualifiedName(tables, identifier.name);
if (table)
column_name = identifier.name;
}
}
}
if (!table)
throw Exception("Cannot find column " + identifier.name, ErrorCodes::THERE_IS_NO_COLUMN);
AnalyzeColumns::ColumnInfo info;
info.node = ast;
info.table = *table;
info.name_in_table = column_name;
if (table->structure_of_subquery)
{
if (!table->structure_of_subquery.has(column_name))
throw Exception("Cannot find column " + backQuoteIfNeed(column_name) + " in subquery", ErrorCodes::LOGICAL_ERROR);
info.data_type = table->structure_of_subquery.getByName(column_name).type;
}
else if (table->storage)
{
info.data_type = table->storage->getColumn(column_name).type;
}
else
throw Exception("Logical error: no storage and no structure of subquery is specified for table", ErrorCodes::LOGICAL_ERROR);
columns[identifier.name] = info;
}
void processImpl(ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables)
{
/// Don't go into subqueries and table-like expressions.
if (typeid_cast<const ASTSelectQuery *>(ast.get())
|| typeid_cast<const ASTTableExpression *>(ast.get()))
{
return;
}
else if (const ASTFunction * func = typeid_cast<const ASTFunction *>(ast.get()))
{
String func_name_lowercase = Poco::toLower(func->name);
/// As special case, treat count(*) as count(), not as count(list of all columns).
if (func_name_lowercase == "count" && func->arguments->children.size() == 1
&& typeid_cast<const ASTAsterisk *>(func->arguments->children[0].get()))
{
func->arguments->children.clear();
}
}
else if (typeid_cast<ASTExpressionList *>(ast.get()))
{
/// Replace asterisks to list of columns.
ASTs & asts = ast->children;
for (int i = static_cast<int>(asts.size()) - 1; i >= 0; --i)
{
if (typeid_cast<ASTAsterisk *>(asts[i].get()))
{
ASTs expanded = expandUnqualifiedAsterisk(tables);
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, expanded.begin(), expanded.end());
}
else if (ASTQualifiedAsterisk * asterisk = typeid_cast<ASTQualifiedAsterisk *>(asts[i].get()))
{
ASTs expanded = expandQualifiedAsterisk(*asterisk, tables);
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, expanded.begin(), expanded.end());
}
}
}
else if (typeid_cast<const ASTIdentifier *>(ast.get()))
{
processIdentifier(ast, columns, aliases, tables);
return;
}
for (auto & child : ast->children)
processImpl(child, columns, aliases, tables);
}
}
void AnalyzeColumns::process(ASTPtr & ast, const CollectAliases & aliases, const CollectTables & tables)
{
/// If this is SELECT query, don't go into FORMAT and SETTINGS clauses
/// - they contain identifiers that are not columns.
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
for (auto & child : ast->children)
{
if (select && child.get() == select->settings.get())
continue;
processImpl(child, columns, aliases, tables);
}
}
void AnalyzeColumns::dump(WriteBuffer & out) const
{
/// For need of tests, we need to dump result in some fixed order.
std::vector<Columns::const_iterator> vec;
vec.reserve(columns.size());
for (auto it = columns.begin(); it != columns.end(); ++it)
vec.emplace_back(it);
std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; });
for (const auto & it : vec)
{
writeString(it->first, out);
writeCString(" -> ", out);
writeProbablyBackQuotedString(it->second.name_in_table, out);
writeCString(" ", out);
writeProbablyBackQuotedString(it->second.data_type->getName(), out);
const auto & table = it->second.table;
writeCString(". Database name: ", out);
if (table.database_name.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.database_name, out);
writeCString(". Table name: ", out);
if (table.table_name.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.table_name, out);
writeCString(". Alias: ", out);
if (table.alias.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.alias, out);
writeCString(". Storage: ", out);
if (!table.storage)
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.storage->getName(), out);
writeCString(". AST: ", out);
if (it->second.node)
{
std::stringstream formatted_ast;
formatAST(*it->second.node, formatted_ast, false, true);
writeString(formatted_ast.str(), out);
}
else
writeCString("(none)", out);
writeChar('\n', out);
}
}
}

View File

@ -1,48 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Analyzers/CollectTables.h>
#include <unordered_map>
namespace DB
{
class WriteBuffer;
struct CollectAliases;
struct CollectTables;
/** For every identifier, that is not an alias,
* determine from what table it comes,
* its original name in table,
* and its data type.
*
* Also:
* - expand asterisks (such as *, t.*, db.table.* and (TODO) even db.table.nested.*) to corresponding list of columns;
* - translate count(*) to count();
* - TODO expand alias columns that come from table definition;
* - TODO replace column names to fully qualified names: identical columns will have same names.
*
* If column is not found or in case of ambiguity, throw an exception.
*/
struct AnalyzeColumns
{
void process(ASTPtr & ast, const CollectAliases & aliases, const CollectTables & tables);
struct ColumnInfo
{
ASTPtr node;
CollectTables::TableInfo table;
String name_in_table;
DataTypePtr data_type;
};
using Columns = std::unordered_map<String, ColumnInfo>;
Columns columns;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,201 +0,0 @@
#include <vector>
#include <Analyzers/AnalyzeLambdas.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_LAMBDA;
extern const int RESERVED_IDENTIFIER_NAME;
}
AnalyzeLambdas::LambdaParameters AnalyzeLambdas::extractLambdaParameters(ASTPtr & ast)
{
/// Lambda parameters could be specified in AST in two forms:
/// - just as single parameter: x -> x + 1
/// - parameters in tuple: (x, y) -> x + 1
#define LAMBDA_ERROR_MESSAGE " There are two valid forms of lambda expressions: x -> ... and (x, y...) -> ..."
if (!ast->tryGetAlias().empty())
throw Exception("Lambda parameters cannot have aliases."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(ast.get()))
{
return { identifier->name };
}
else if (const ASTFunction * function = typeid_cast<const ASTFunction *>(ast.get()))
{
if (function->name != "tuple")
throw Exception("Left hand side of '->' or first argument of 'lambda' is a function, but this function is not tuple."
LAMBDA_ERROR_MESSAGE " Found function '" + function->name + "' instead.", ErrorCodes::BAD_LAMBDA);
if (!function->arguments || function->arguments->children.empty())
throw Exception("Left hand side of '->' or first argument of 'lambda' is empty tuple."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
LambdaParameters res;
res.reserve(function->arguments->children.size());
for (const ASTPtr & arg : function->arguments->children)
{
const ASTIdentifier * arg_identifier = typeid_cast<const ASTIdentifier *>(arg.get());
if (!arg_identifier)
throw Exception("Left hand side of '->' or first argument of 'lambda' contains something that is not just identifier."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
if (!arg_identifier->children.empty())
throw Exception("Left hand side of '->' or first argument of 'lambda' contains compound identifier."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
if (!arg_identifier->alias.empty())
throw Exception("Lambda parameters cannot have aliases."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
res.emplace_back(arg_identifier->name);
}
return res;
}
else
throw Exception("Unexpected left hand side of '->' or first argument of 'lambda'."
LAMBDA_ERROR_MESSAGE, ErrorCodes::BAD_LAMBDA);
#undef LAMBDA_ERROR_MESSAGE
}
namespace
{
/// Currently visible parameters in all scopes of lambda expressions.
/// Lambda expressions could be nested: arrayMap(x -> arrayMap(y -> x[y], x), [[1], [2, 3]])
using LambdaScopes = std::vector<AnalyzeLambdas::LambdaParameters>;
void processIdentifier(ASTPtr & ast, LambdaScopes & lambda_scopes)
{
ASTIdentifier & identifier = static_cast<ASTIdentifier &>(*ast);
if (identifier.children.empty())
{
bool found = false;
/// From most inner scope towards outer scopes.
for (ssize_t num_scopes = lambda_scopes.size(), scope_idx = num_scopes - 1; scope_idx >= 0; --scope_idx)
{
for (size_t arg_idx = 0, num_args = lambda_scopes[scope_idx].size(); arg_idx < num_args; ++arg_idx)
{
if (lambda_scopes[scope_idx][arg_idx] == identifier.name)
{
identifier.name = "_lambda" + toString(scope_idx) + "_arg" + toString(arg_idx);
found = true;
break;
}
}
if (found)
break;
}
if (!found && startsWith(identifier.name, "_lambda"))
throw Exception("Identifier names starting with '_lambda' are reserved for parameters of lambda expressions.",
ErrorCodes::RESERVED_IDENTIFIER_NAME);
}
}
void processImpl(
ASTPtr & ast,
LambdaScopes & lambda_scopes,
const ASTPtr & parent_function_for_this_argument,
AnalyzeLambdas::HigherOrderFunctions & higher_order_functions)
{
/// Don't go into subqueries and table-like expressions.
if (typeid_cast<const ASTSelectQuery *>(ast.get())
|| typeid_cast<const ASTTableExpression *>(ast.get()))
{
return;
}
else if (ASTFunction * func = typeid_cast<ASTFunction *>(ast.get()))
{
/** We must memoize parameters from left hand side (x, y) and then analyze right hand side.
*/
if (func->name == "lambda")
{
auto num_arguments = func->arguments->children.size();
if (num_arguments != 2)
throw Exception("Lambda expression ('->' or 'lambda' function) must have exactly two arguments."
" Found " + toString(num_arguments) + " instead.", ErrorCodes::BAD_LAMBDA);
lambda_scopes.emplace_back(AnalyzeLambdas::extractLambdaParameters(func->arguments->children[0]));
for (size_t i = 0; i < num_arguments; ++i)
processImpl(func->arguments->children[i], lambda_scopes, nullptr, higher_order_functions);
lambda_scopes.pop_back();
if (!parent_function_for_this_argument)
throw Exception("Lambda expression ('->' or 'lambda' function) must be presented as an argument of higher-order function."
" Found standalone lambda expression instead.", ErrorCodes::BAD_LAMBDA);
higher_order_functions.emplace_back(parent_function_for_this_argument);
}
else
{
/// When diving into function arguments, pass current ast node.
if (func->arguments)
for (auto & child : func->arguments->children)
processImpl(child, lambda_scopes, ast, higher_order_functions);
if (func->parameters)
for (auto & child : func->parameters->children)
processImpl(child, lambda_scopes, nullptr, higher_order_functions);
}
return;
}
else if (typeid_cast<ASTIdentifier *>(ast.get()))
{
processIdentifier(ast, lambda_scopes);
return;
}
for (auto & child : ast->children)
processImpl(child, lambda_scopes, nullptr, higher_order_functions);
}
}
void AnalyzeLambdas::process(ASTPtr & ast)
{
LambdaScopes lambda_scopes;
for (auto & child : ast->children)
processImpl(child, lambda_scopes, nullptr, higher_order_functions);
}
void AnalyzeLambdas::dump(WriteBuffer & out) const
{
for (const auto & ast : higher_order_functions)
{
writeString(ast->getColumnName(), out);
writeChar('\n', out);
}
}
}

View File

@ -1,35 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <vector>
namespace DB
{
class WriteBuffer;
/** For every lambda expression, rename its parameters to '_lambda0_arg0' form.
* Check correctness of lambda expressions.
* Find functions, that have lambda expressions as arguments (they are called "higher order" functions).
*
* This should be done before CollectAliases.
*/
struct AnalyzeLambdas
{
void process(ASTPtr & ast);
/// Parameters of lambda expressions.
using LambdaParameters = std::vector<String>;
static LambdaParameters extractLambdaParameters(ASTPtr & ast);
using HigherOrderFunctions = std::vector<ASTPtr>;
HigherOrderFunctions higher_order_functions;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,70 +0,0 @@
#include <Analyzers/AnalyzeResultOfQuery.h>
#include <Analyzers/CollectAliases.h>
#include <Analyzers/CollectTables.h>
#include <Analyzers/AnalyzeLambdas.h>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/TypeAndConstantInference.h>
#include <Interpreters/Context.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTSelectQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_AST_STRUCTURE;
}
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)
throw Exception("AnalyzeResultOfQuery::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (!select->select_expression_list)
throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
AnalyzeLambdas analyze_lambdas;
analyze_lambdas.process(ast);
CollectAliases collect_aliases;
collect_aliases.process(ast);
CollectTables collect_tables;
collect_tables.process(ast, context, collect_aliases, table_functions);
AnalyzeColumns analyze_columns;
analyze_columns.process(ast, collect_aliases, collect_tables);
TypeAndConstantInference inference;
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, table_functions);
for (const ASTPtr & child : select->select_expression_list->children)
{
auto it = inference.info.find(child->getColumnName());
if (it == inference.info.end())
throw Exception("Logical error: type information for result column of SELECT query was not inferred", ErrorCodes::LOGICAL_ERROR);
String name = child->getAliasOrColumnName();
const TypeAndConstantInference::ExpressionInfo & info = it->second;
result.insert(ColumnWithTypeAndName(
info.is_constant_expression ? info.data_type->createColumnConst(1, info.value) : nullptr,
info.data_type,
std::move(name)));
}
}
void AnalyzeResultOfQuery::dump(WriteBuffer & out) const
{
writeString(result.dumpStructure(), out);
}
}

View File

@ -1,32 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Core/Block.h>
namespace DB
{
class WriteBuffer;
class Context;
struct ExecuteTableFunctions;
/** For SELECT query, determine names and types of columns of result,
* and if some columns are constant expressions, calculate their values.
*
* NOTE It's possible to memoize calculations, that happens under the hood
* and could be duplicated in subsequent analysis of subqueries.
*/
struct AnalyzeResultOfQuery
{
void process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions);
/// Block will have non-nullptr columns for constant expressions.
Block result;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,3 +0,0 @@
if(ENABLE_TESTS)
add_subdirectory(tests)
endif()

View File

@ -1,109 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS;
}
static void processImpl(const ASTPtr & ast, CollectAliases::Aliases & aliases, CollectAliases::Kind kind, size_t keep_kind_for_depth)
{
String alias = ast->tryGetAlias();
if (!alias.empty())
{
auto it_inserted = aliases.emplace(alias, CollectAliases::AliasInfo(ast, kind));
if (!it_inserted.second && ast->getTreeHash() != it_inserted.first->second.node->getTreeHash())
{
std::stringstream message;
message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":\n";
formatAST(*it_inserted.first->second.node, message, false, true);
message << "\nand\n";
formatAST(*ast, message, false, true);
message << "\n";
throw Exception(message.str(), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
}
}
for (auto & child : ast->children)
{
if (typeid_cast<const ASTSelectQuery *>(child.get()))
{
/// Don't go into subqueries.
}
else if (typeid_cast<const ASTTableExpression *>(child.get()))
{
processImpl(child, aliases, CollectAliases::Kind::Table, 1);
}
else if (typeid_cast<const ASTArrayJoin *>(child.get()))
{
/// ASTArrayJoin -> ASTExpressionList -> element of expression AS alias
processImpl(child, aliases, CollectAliases::Kind::ArrayJoin, 3);
}
else if (keep_kind_for_depth > 0)
{
processImpl(child, aliases, kind, keep_kind_for_depth - 1);
}
else
{
processImpl(child, aliases, CollectAliases::Kind::Expression, 0);
}
}
}
void CollectAliases::process(const ASTPtr & ast)
{
processImpl(ast, aliases, Kind::Expression, 0);
}
void CollectAliases::dump(WriteBuffer & out) const
{
/// For need of tests, we need to dump result in some fixed order.
std::vector<Aliases::const_iterator> vec;
vec.reserve(aliases.size());
for (auto it = aliases.begin(); it != aliases.end(); ++it)
vec.emplace_back(it);
std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; });
for (const auto & it : vec)
{
writeProbablyBackQuotedString(it->first, out);
writeCString(" -> ", out);
switch (it->second.kind)
{
case Kind::Expression:
writeCString("(expression) ", out);
break;
case Kind::Table:
writeCString("(table) ", out);
break;
case Kind::ArrayJoin:
writeCString("(array join) ", out);
break;
}
std::stringstream formatted_ast;
formatAST(*it->second.node, formatted_ast, false, true);
writeString(formatted_ast.str(), out);
writeChar('\n', out);
}
}
}

View File

@ -1,54 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <unordered_map>
namespace DB
{
class WriteBuffer;
/** Build a map: alias -> AST node.
*
* Also fill information about what kind each alias has:
* - expression alias;
* - table alias;
* - ARRAY JOIN alias.
*
* As extension to standard SQL, aliases could be specified and used in any part of query.
* Example: SELECT a, (1 AS a) + 1 AS b FROM t GROUP BY a, b
* Alias could be used in query before it is defined.
*
* Aliases could not be redefined. Example: 1 AS a, a + 1 AS a - is prohibited.
*
* Don't descend into subqueries (as aliases are local inside them).
*/
struct CollectAliases
{
void process(const ASTPtr & ast);
enum class Kind
{
Expression, /// Example: SELECT a AS b, f(x) AS y
Table, /// Example: SELECT t.* FROM (SELECT 1) AS t
ArrayJoin /// Example: SELECT t.x.a FROM t ARRAY JOIN arr AS x
};
struct AliasInfo
{
ASTPtr node;
Kind kind;
AliasInfo(const ASTPtr & node, Kind kind) : node(node), kind(kind) {}
};
using Aliases = std::unordered_map<String, AliasInfo>;
Aliases aliases;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,186 +0,0 @@
#include <Analyzers/CollectTables.h>
#include <Analyzers/CollectAliases.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Analyzers/AnalyzeResultOfQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSubquery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int LOGICAL_ERROR;
}
static CollectTables::TableInfo processOrdinaryTable(const ASTPtr & ast_database_and_table_name, const Context & context)
{
const ASTIdentifier & identifier = static_cast<const ASTIdentifier &>(*ast_database_and_table_name);
CollectTables::TableInfo res;
res.node = ast_database_and_table_name;
res.alias = identifier.tryGetAlias();
if (ast_database_and_table_name->children.empty())
{
res.table_name = identifier.name;
}
else
{
if (ast_database_and_table_name->children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
res.database_name = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
res.table_name = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
res.storage = context.getTable(res.database_name, res.table_name);
return res;
}
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const ExecuteTableFunctions & table_functions)
{
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
IAST::Hash ast_hash = ast_table_function->getTreeHash();
auto it = table_functions.tables.find(ast_hash);
if (table_functions.tables.end() == it)
throw Exception("Table function " + function.name + " was not executed in advance.", ErrorCodes::LOGICAL_ERROR);
CollectTables::TableInfo res;
res.node = ast_table_function;
res.alias = function.tryGetAlias();
res.storage = it->second;
return res;
}
static CollectTables::TableInfo processNoTables(const Context & context)
{
/// No FROM section. Interpret it as FROM system.one.
CollectTables::TableInfo res;
res.database_name = "system";
res.table_name = "one";
res.storage = context.getTable(res.database_name, res.table_name);
return res;
}
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context, ExecuteTableFunctions & table_functions)
{
AnalyzeResultOfQuery analyzer;
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context, table_functions);
CollectTables::TableInfo res;
res.node = ast_subquery;
res.alias = ast_subquery->tryGetAlias();
res.structure_of_subquery = analyzer.result;
return res;
}
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & /*aliases*/, ExecuteTableFunctions & table_functions)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)
throw Exception("CollectTables::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (!select->tables)
{
tables.emplace_back(processNoTables(context));
return;
}
for (auto & child : select->tables->children)
{
ASTTablesInSelectQueryElement & element = static_cast<ASTTablesInSelectQueryElement &>(*child);
if (!element.table_expression) /// This is ARRAY JOIN
continue;
ASTTableExpression & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
if (table_expression.database_and_table_name)
{
tables.emplace_back(processOrdinaryTable(table_expression.database_and_table_name, context));
/// TODO It could be alias to another table expression.
}
else if (table_expression.table_function)
{
tables.emplace_back(processTableFunction(table_expression.table_function, table_functions));
}
else if (table_expression.subquery)
{
tables.emplace_back(processSubquery(table_expression.subquery, context, table_functions));
}
else
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
}
/// TODO Control that tables don't have conflicting names.
}
void CollectTables::dump(WriteBuffer & out) const
{
for (const auto & table : tables)
{
writeCString("Database name: ", out);
if (table.database_name.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.database_name, out);
writeCString(". Table name: ", out);
if (table.table_name.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.table_name, out);
writeCString(". Alias: ", out);
if (table.alias.empty())
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.alias, out);
writeCString(". Storage: ", out);
if (!table.storage)
writeCString("(none)", out);
else
writeProbablyBackQuotedString(table.storage->getName(), out);
writeCString(". Structure of subquery: ", out);
if (!table.structure_of_subquery)
writeCString("(none)", out);
else
writeString(table.structure_of_subquery.dumpStructure(), out);
writeCString(". AST: ", out);
if (!table.node)
writeCString("(none)", out);
else
{
std::stringstream formatted_ast;
formatAST(*table.node, formatted_ast, false, true);
writeString(formatted_ast.str(), out);
}
writeChar('\n', out);
}
}
}

View File

@ -1,52 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Storages/IStorage.h>
#include <vector>
namespace DB
{
class Context;
struct CollectAliases;
struct ExecuteTableFunctions;
class WriteBuffer;
/** Collect and analyze table-like expressions in section FROM in a query.
* For every expression, keep its alias.
*
* For ordinary tables, determine database and table name, obtain and keep StoragePtr.
* For subqueries, determine result structure. This requires analysis of subquery, such as type inference.
* For table functions, grab them from prepared ExecuteTableFunctions object.
*/
struct CollectTables
{
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions);
enum class Kind
{
OrdinaryTable,
TableFunction,
Subquery
};
struct TableInfo
{
ASTPtr node;
String database_name;
String table_name;
String alias;
StoragePtr storage;
Block structure_of_subquery;
};
using Tables = std::vector<TableInfo>;
Tables tables;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,80 +0,0 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
#include <Analyzers/ExecuteTableFunctions.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int LOGICAL_ERROR;
}
/// Allows to execute exactly same table functions only once.
using ASTTreeToTable = std::map<IAST::Hash, StoragePtr>;
static void processTableFunction(const ASTPtr & ast_table_function, const Context & context, ExecuteTableFunctions::Tables & result_map)
{
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
/// If already executed.
IAST::Hash ast_hash = ast_table_function->getTreeHash();
if (result_map.count(ast_hash))
return;
/// Obtain table function
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(function.name, context);
/// Execute it and store result
StoragePtr table = table_function_ptr->execute(ast_table_function, context);
result_map[ast_hash] = table;
}
void ExecuteTableFunctions::process(ASTPtr & ast, const Context & context)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)
throw Exception("ExecuteTableFunctions::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (!select->tables)
return;
for (auto & child : select->tables->children)
{
ASTTablesInSelectQueryElement & element = static_cast<ASTTablesInSelectQueryElement &>(*child);
if (!element.table_expression) /// This is ARRAY JOIN
continue;
ASTTableExpression & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
if (!table_expression.table_function)
continue;
processTableFunction(table_expression.table_function, context, tables);
}
}
void ExecuteTableFunctions::dump(WriteBuffer & out) const
{
for (const auto & table : tables)
{
writeString(table.second->getName(), out);
writeCString("\n\n", out);
writeString(table.second->getColumns().getAllPhysical().toString(), out);
writeCString("\n", out);
}
}
}

View File

@ -1,35 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Storages/IStorage.h>
#include <map>
namespace DB
{
class Context;
class WriteBuffer;
/** For every table function, found in first level of the query
* (don't go into subqueries)
* execute it and save corresponding StoragePtr.
*
* Execution of table functions must be done in a stage of query analysis,
* because otherwise we don't know table structure. So, it is assumed as cheap operation.
*
* Multiple occurences of table functions with same arguments will be executed only once.
*/
struct ExecuteTableFunctions
{
void process(ASTPtr & ast, const Context & context);
using Tables = std::map<IAST::Hash, StoragePtr>;
Tables tables;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,264 +0,0 @@
#include <set>
#include <unordered_set>
#include <Analyzers/OptimizeGroupOrderLimitBy.h>
#include <Analyzers/TypeAndConstantInference.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTLiteral.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_AST_STRUCTURE;
}
static bool isInjectiveFunction(
const ASTFunction * ast_function,
const TypeAndConstantInference::ExpressionInfo & function_info,
const TypeAndConstantInference::Info & all_info)
{
if (!function_info.function)
return false;
Block block_with_constants;
const ASTs & children = ast_function->arguments->children;
for (const auto & child : children)
{
String child_name = child->getColumnName();
const TypeAndConstantInference::ExpressionInfo & child_info = all_info.at(child_name);
block_with_constants.insert(ColumnWithTypeAndName(
child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr,
child_info.data_type,
child_name));
}
return function_info.function->isInjective(block_with_constants);
}
static bool isDeterministicFunctionOfKeys(
const ASTFunction * ast_function,
const TypeAndConstantInference::ExpressionInfo & function_info,
const TypeAndConstantInference::Info & all_info,
const std::vector<std::string> & keys)
{
if (!function_info.function || !function_info.function->isDeterministicInScopeOfQuery())
return false;
for (const auto & child : ast_function->arguments->children)
{
String child_name = child->getColumnName();
const TypeAndConstantInference::ExpressionInfo & child_info = all_info.at(child_name);
/// Function argument is constant.
if (child_info.is_constant_expression)
continue;
/// Function argument is one of keys.
if (keys.end() != std::find(keys.begin(), keys.end(), child_name))
continue;
/// Function argument is a function, that deterministically depend on keys.
if (const ASTFunction * child_function = typeid_cast<const ASTFunction *>(child.get()))
{
if (isDeterministicFunctionOfKeys(child_function, child_info, all_info, keys))
continue;
}
return false;
}
return true;
}
static void processGroupByLikeList(ASTPtr & ast, TypeAndConstantInference & expression_info)
{
if (!ast)
return;
ASTs & elems = ast->children;
std::unordered_set<std::string> unique_keys;
size_t i = 0;
auto restart = [&]
{
i = 0;
unique_keys.clear();
};
/// Always leave last element in GROUP BY, even if it is constant.
while (i < elems.size() && elems.size() > 1)
{
ASTPtr & elem = elems[i];
String column_name = elem->getColumnName(); /// TODO canonicalization of names
auto it = expression_info.info.find(column_name);
if (it == expression_info.info.end())
throw Exception("Type inference was not done for " + column_name, ErrorCodes::LOGICAL_ERROR);
const TypeAndConstantInference::ExpressionInfo & info = it->second;
/// Removing constant expressions.
/// Removing duplicate keys.
if (info.is_constant_expression
|| !unique_keys.emplace(column_name).second)
{
elems.erase(elems.begin() + i);
continue;
}
if (info.function && !elem->children.empty())
{
const ASTFunction * ast_function = typeid_cast<const ASTFunction *>(elem.get());
if (!ast_function)
throw Exception("Column is marked as function during type inference, but corresponding AST node "
+ column_name + " is not a function", ErrorCodes::LOGICAL_ERROR);
/// Unwrap injective functions.
if (isInjectiveFunction(ast_function, info, expression_info.info))
{
auto args = ast_function->arguments;
elems.erase(elems.begin() + i);
elems.insert(elems.begin() + i, args->children.begin(), args->children.end());
restart(); /// Previous keys may become deterministic function of newly added keys.
continue;
}
/// Remove deterministic functions of another keys.
std::vector<String> other_keys;
other_keys.reserve(elems.size() - 1);
for (size_t j = 0, size = elems.size(); j < size; ++j)
if (j != i)
other_keys.emplace_back(elems[j]->getColumnName());
if (isDeterministicFunctionOfKeys(ast_function, info, expression_info.info, other_keys))
{
elems.erase(elems.begin() + i);
continue;
}
}
++i;
}
}
static void processOrderByList(ASTPtr & ast, TypeAndConstantInference & expression_info)
{
if (!ast)
return;
ASTs & elems = ast->children;
/// sort column name and collation
std::set<std::pair<std::string, std::string>> unique_keys;
size_t i = 0;
while (i < elems.size())
{
const ASTOrderByElement * order_by_elem = typeid_cast<const ASTOrderByElement *>(elems[i].get());
if (!order_by_elem)
throw Exception("Child of ORDER BY clause is not an ASTOrderByElement", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// It has ASC|DESC and COLLATE inplace, and expression as its only child.
if (order_by_elem->children.empty())
throw Exception("ORDER BY element has no children", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const ASTPtr & elem = order_by_elem->children[0];
String collation;
if (order_by_elem->collation)
{
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(order_by_elem->collation.get());
if (!lit)
throw Exception("Collation in ORDER BY clause is not an ASTLiteral", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (lit->value.getType() != Field::Types::String)
throw Exception("Collation in ORDER BY clause is not a string literal", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
collation = lit->value.get<String>();
}
String column_name = elem->getColumnName();
auto it = expression_info.info.find(column_name);
if (it == expression_info.info.end())
throw Exception("Type inference was not done for " + column_name, ErrorCodes::LOGICAL_ERROR);
const TypeAndConstantInference::ExpressionInfo & info = it->second;
/// Removing constant expressions.
/// Removing duplicate keys.
if (info.is_constant_expression
/// Having same element but with empty collation. Empty collation is considered more "granular" than any special collation.
|| unique_keys.count(std::make_pair(column_name, String()))
/// Having same element with same collation.
|| !unique_keys.emplace(column_name, collation).second)
{
elems.erase(elems.begin() + i);
continue;
}
if (i > 0 && collation.empty() && info.function && !elem->children.empty())
{
const ASTFunction * ast_function = typeid_cast<const ASTFunction *>(elem.get());
if (!ast_function)
throw Exception("Column is marked as function during type inference, but corresponding AST node "
+ column_name + " is not a function", ErrorCodes::LOGICAL_ERROR);
/// Remove deterministic functions of previous keys. Only consider keys without collation.
std::vector<String> prev_keys;
prev_keys.reserve(i);
for (size_t j = 0; j < i; ++j)
if (!typeid_cast<const ASTOrderByElement &>(*elems[j]).collation)
prev_keys.emplace_back(elems[j]->children.at(0)->getColumnName());
if (isDeterministicFunctionOfKeys(ast_function, info, expression_info.info, prev_keys))
{
elems.erase(elems.begin() + i);
continue;
}
}
++i;
}
}
void OptimizeGroupOrderLimitBy::process(ASTPtr & ast, TypeAndConstantInference & expression_info)
{
ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(ast.get());
if (!select)
throw Exception("AnalyzeResultOfQuery::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (!select->select_expression_list)
throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
processGroupByLikeList(select->group_expression_list, expression_info);
processGroupByLikeList(select->limit_by_expression_list, expression_info);
if (select->order_expression_list)
{
processOrderByList(select->order_expression_list, expression_info);
/// ORDER BY could be completely eliminated
if (select->order_expression_list->children.empty())
{
select->children.erase(std::remove(
select->children.begin(), select->children.end(), select->order_expression_list), select->children.end());
select->order_expression_list.reset();
}
}
}
}

View File

@ -1,36 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
class Context;
class WriteBuffer;
struct TypeAndConstantInference;
/** Transform GROUP BY, ORDER BY and LIMIT BY sections.
* (LIMIT BY is an extension to SQL language, do not be confused with ordinary LIMIT)
*
* Remove constant expressions (like ORDER BY concat('hello', 'world')).
* For GROUP BY, unwrap injective functions (like GROUP BY toString(x) -> GROUP BY x).
* For GROUP BY, remove deterministic functions of another keys (like GROUP BY x + 1, x -> GROUP BY x).
* TODO For ORDER BY, remove deterministic functions of previous keys (like ORDER BY num, toString(num) -> ORDER BY num),
* but only if no collation has specified.
* As a special case, remove duplicate keys.
* For LIMIT BY, apply all the same as for GROUP BY.
*
* TODO We should apply something similar for DISTINCT,
* but keys for DISTINCT are specified implicitly (as whole SELECT expression list).
*
* This should be run after CollectAliases, because some aliases will be lost from AST during this transformation.
* This should be run after TranslatePositionalArguments for positional arguments like ORDER BY 1, 2 not to be confused with constants.
*/
struct OptimizeGroupOrderLimitBy
{
void process(ASTPtr & ast, TypeAndConstantInference & expression_info);
};
}

View File

@ -1,86 +0,0 @@
#include <Analyzers/TranslatePositionalArguments.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE;
}
static void processElement(ASTPtr & element, const ASTPtr & select_expression_list, const char * description)
{
if (ASTLiteral * literal = typeid_cast<ASTLiteral *>(element.get()))
{
if (literal->value.getType() == Field::Types::UInt64)
{
UInt64 position = literal->value.get<UInt64>();
if (!literal->alias.empty())
throw Exception("Unsigned numeric literal " + toString(position) + " in " + toString(description)
+ " section is interpreted as positional argument, "
"but it has alias " + backQuoteIfNeed(literal->alias) + " that is not expected", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (position == 0)
throw Exception("Unsigned numeric literal 0 in " + toString(description) + " section is interpreted as positional argument, "
"but positional arguments are 1-based", ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE);
if (position > select_expression_list->children.size())
throw Exception("Unsigned numeric literal " + toString(position) + " in " + String(description)
+ " section is interpreted as positional argument, "
"but it is greater than number of expressions in SELECT section ("
+ toString(select_expression_list->children.size()) + ")", ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE);
element = select_expression_list->children[position - 1]->clone();
}
}
}
static void processClause(ASTPtr & ast, const ASTPtr & select_expression_list, const char * description, bool is_order_by)
{
if (!ast)
return;
for (auto & child : ast->children)
{
if (is_order_by)
{
if (!typeid_cast<ASTOrderByElement *>(child.get()))
throw Exception("Child of ORDER BY clause is not an ASTOrderByElement", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// It has ASC|DESC and COLLATE inplace, and expression as its only child.
if (child->children.empty())
throw Exception("ORDER BY element has no children", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
processElement(child->children[0], select_expression_list, description);
}
else
processElement(child, select_expression_list, description);
}
}
void TranslatePositionalArguments::process(ASTPtr & ast)
{
ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(ast.get());
if (!select)
throw Exception("TranslatePositionalArguments::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
if (!select->select_expression_list)
throw Exception("SELECT query doesn't have select_expression_list", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
processClause(select->group_expression_list, select->select_expression_list, "GROUP BY", false);
processClause(select->order_expression_list, select->select_expression_list, "ORDER BY", true);
processClause(select->limit_by_expression_list, select->select_expression_list, "LIMIT BY", false);
}
}

View File

@ -1,21 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
class Context;
class WriteBuffer;
/** Transform GROUP BY, ORDER BY and LIMIT BY sections.
* Replace positional arguments (like ORDER BY 1, 2) to corresponding columns.
*/
struct TranslatePositionalArguments
{
void process(ASTPtr & ast);
};
}

View File

@ -1,534 +0,0 @@
#include <Poco/String.h>
#include <Analyzers/TypeAndConstantInference.h>
#include <Analyzers/CollectAliases.h>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/AnalyzeLambdas.h>
#include <Analyzers/AnalyzeResultOfQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/convertFieldToType.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSubquery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeFunction.h>
#include <algorithm>
#include <sstream>
#include <unordered_map>
#include <vector>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int BAD_LAMBDA;
}
namespace
{
Field getValueFromConstantColumn(const ColumnPtr & column)
{
if (!column->isColumnConst())
throw Exception("Logical error: expected that column is constant", ErrorCodes::LOGICAL_ERROR);
if (column->size() != 1)
throw Exception("Logical error: expected that column with constant has single element", ErrorCodes::LOGICAL_ERROR);
return (*column)[0];
}
/// Description of single parameter of lambda expression: name -> type.
/// Note, that after AnalyzeLambda step, names of lambda parameters are unique in single select query.
using LambdaParameters = std::unordered_map<String, DataTypePtr>;
void processImpl(
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas,
ExecuteTableFunctions & table_functions);
void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info)
{
const ASTLiteral * literal = static_cast<const ASTLiteral *>(ast.get());
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.is_constant_expression = true;
expression_info.data_type = applyVisitor(FieldToDataType(), literal->value);
expression_info.value = convertFieldToType(literal->value, *expression_info.data_type);
info.emplace(column_name, std::move(expression_info));
}
void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info,
const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
const AnalyzeLambdas & lambdas, ExecuteTableFunctions & table_functions)
{
/// Column from table
auto it = columns.columns.find(column_name);
if (it != columns.columns.end())
{
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.data_type = it->second.data_type;
/// If it comes from subquery and we know, that it is constant expression.
const Block & structure_of_subquery = it->second.table.structure_of_subquery;
if (structure_of_subquery)
{
const ColumnWithTypeAndName & column_from_subquery = structure_of_subquery.getByName(it->second.name_in_table);
if (column_from_subquery.column)
{
expression_info.is_constant_expression = true;
expression_info.value = getValueFromConstantColumn(column_from_subquery.column);
}
}
info.emplace(column_name, std::move(expression_info));
}
else
{
/// Alias
auto jt = aliases.aliases.find(column_name);
if (jt != aliases.aliases.end())
{
/// TODO Cyclic aliases.
if (jt->second.kind != CollectAliases::Kind::Expression)
throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR);
processImpl(jt->second.node, context, aliases, columns, info, lambdas, table_functions);
info[column_name] = info[jt->second.node->getColumnName()];
}
}
}
void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
const Context & context)
{
ASTFunction * function = static_cast<ASTFunction *>(ast.get());
/// Special case for lambda functions. Lambda function has special return type "Function".
/// We first create info with Function of unspecified arguments, and will specify them later.
if (function->name == "lambda")
{
size_t number_of_lambda_parameters = AnalyzeLambdas::extractLambdaParameters(function->arguments->children.at(0)).size();
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.data_type = std::make_unique<DataTypeFunction>(DataTypes(number_of_lambda_parameters));
info.emplace(column_name, std::move(expression_info));
return;
}
DataTypes argument_types;
ColumnsWithTypeAndName argument_columns;
if (function->arguments)
{
for (const auto & child : function->arguments->children)
{
auto it = info.find(child->getColumnName());
if (it == info.end())
throw Exception("Logical error: type of function argument was not inferred during depth-first search", ErrorCodes::LOGICAL_ERROR);
argument_types.emplace_back(it->second.data_type);
argument_columns.emplace_back(ColumnWithTypeAndName(nullptr, it->second.data_type, ""));
if (it->second.is_constant_expression)
argument_columns.back().column = it->second.data_type->createColumnConst(1, it->second.value);
}
}
/// Special cases for COUNT(DISTINCT ...) function.
bool column_name_changed = false;
String func_name_lowercase = Poco::toLower(function->name);
if (func_name_lowercase == "countdistinct") /// It comes in that form from parser.
{
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
function->name = context.getSettingsRef().count_distinct_implementation;
column_name_changed = true;
}
/// Aggregate function.
Array parameters = (function->parameters) ? getAggregateFunctionParametersArray(function->parameters) : Array();
if (AggregateFunctionPtr aggregate_function_ptr = AggregateFunctionFactory::instance().tryGet(function->name, argument_types, parameters))
{
/// Note that aggregate function could never be constant expression.
/// (?) Replace function name to canonical one. Because same function could be referenced by different names.
// function->name = aggregate_function_ptr->getName();
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.data_type = aggregate_function_ptr->getReturnType();
expression_info.aggregate_function = aggregate_function_ptr;
info.emplace(column_name_changed ? ast->getColumnName() : column_name, std::move(expression_info));
return;
}
/// Ordinary function.
if (function->parameters)
throw Exception("The only parametric functions (functions with two separate parenthesis pairs) are aggregate functions"
", and '" + function->name + "' is not an aggregate function.", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
/// IN operator. This is special case, because subqueries in right hand side are not scalar subqueries.
if (function->name == "in"
|| function->name == "notIn"
|| function->name == "globalIn"
|| function->name == "globalNotIn")
{
/// For simplicity reasons, do not consider this as constant expression. We may change it in future.
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.data_type = std::make_shared<DataTypeUInt8>();
info.emplace(column_name, std::move(expression_info));
return;
}
const auto & function_builder_ptr = FunctionFactory::instance().get(function->name, context);
/// (?) Replace function name to canonical one. Because same function could be referenced by different names.
// function->name = function_ptr->getName();
ColumnsWithTypeAndName columns_for_analysis;
columns_for_analysis.reserve(argument_types.size());
bool all_consts = true;
if (function->arguments)
{
for (const auto & child : function->arguments->children)
{
String child_name = child->getColumnName();
const TypeAndConstantInference::ExpressionInfo & child_info = info.at(child_name);
columns_for_analysis.emplace_back(
child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr,
child_info.data_type,
child_name);
if (!child_info.is_constant_expression)
all_consts = false;
}
}
auto function_ptr = function_builder_ptr->build(argument_columns);
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
expression_info.function = function_ptr;
expression_info.data_type = function_ptr->getReturnType();
if (all_consts && function_ptr->isSuitableForConstantFolding())
{
Block block_with_constants(columns_for_analysis);
ColumnNumbers argument_numbers(columns_for_analysis.size());
for (size_t i = 0, size = argument_numbers.size(); i < size; ++i)
argument_numbers[i] = i;
size_t result_position = argument_numbers.size();
block_with_constants.insert({nullptr, expression_info.data_type, column_name});
function_ptr->execute(block_with_constants, argument_numbers, result_position, 1);
const auto & result_column = block_with_constants.getByPosition(result_position).column;
if (result_column->isColumnConst())
{
expression_info.is_constant_expression = true;
expression_info.value = (*result_column)[0];
}
}
info.emplace(column_name, std::move(expression_info));
}
void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
const Context & context, ExecuteTableFunctions & table_functions)
{
ASTSubquery * subquery = static_cast<ASTSubquery *>(ast.get());
AnalyzeResultOfQuery analyzer;
analyzer.process(subquery->children.at(0), context, table_functions);
if (!analyzer.result)
throw Exception("Logical error: no columns returned from scalar subquery", ErrorCodes::LOGICAL_ERROR);
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = ast;
if (analyzer.result.columns() == 1)
{
const auto & elem = analyzer.result.getByPosition(0);
expression_info.data_type = elem.type;
if (elem.column)
{
expression_info.is_constant_expression = true;
expression_info.value = getValueFromConstantColumn(elem.column);
}
}
else
{
/// Result of scalar subquery is interpreted as tuple.
size_t size = analyzer.result.columns();
DataTypes types;
types.reserve(size);
bool all_consts = true;
for (size_t i = 0; i < size; ++i)
{
const auto & elem = analyzer.result.getByPosition(i);
types.emplace_back(elem.type);
if (!elem.column)
all_consts = false;
}
expression_info.data_type = std::make_shared<DataTypeTuple>(types);
if (all_consts)
{
TupleBackend value(size);
for (size_t i = 0; i < size; ++i)
value[i] = getValueFromConstantColumn(analyzer.result.getByPosition(i).column);
expression_info.is_constant_expression = true;
expression_info.value = Tuple(std::move(value));
}
}
info.emplace(column_name, std::move(expression_info));
}
void processHigherOrderFunction(
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas,
ExecuteTableFunctions & table_functions)
{
ASTFunction * function = static_cast<ASTFunction *>(ast.get());
const auto & function_builder_ptr = FunctionFactory::instance().get(function->name, context);
if (!function->arguments)
throw Exception("Unexpected AST for higher-order function", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
DataTypes types;
types.reserve(function->arguments->children.size());
for (const auto & child : function->arguments->children)
{
String child_name = child->getColumnName();
const TypeAndConstantInference::ExpressionInfo & child_info = info.at(child_name);
types.emplace_back(child_info.data_type);
}
function_builder_ptr->getLambdaArgumentTypes(types);
/// For every lambda expression, dive into it.
if (types.size() != function->arguments->children.size())
throw Exception("Logical error: size of types was changed after call to IFunction::getLambdaArgumentTypes",
ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0, size = function->arguments->children.size(); i < size; ++i)
{
const auto & child = function->arguments->children[i];
const ASTFunction * lambda = typeid_cast<const ASTFunction *>(child.get());
if (lambda && lambda->name == "lambda")
{
const auto * lambda_type = typeid_cast<const DataTypeFunction *>(types[i].get());
if (!lambda_type)
throw Exception("Logical error: IFunction::getLambdaArgumentTypes returned data type for lambda expression,"
" that is not DataTypeFunction", ErrorCodes::LOGICAL_ERROR);
if (!lambda->arguments || lambda->arguments->children.size() != 2)
throw Exception("Lambda function must have exactly two arguments (sides of arrow)", ErrorCodes::BAD_LAMBDA);
/// Set types for every lambda parameter.
AnalyzeLambdas::LambdaParameters parameters = AnalyzeLambdas::extractLambdaParameters(lambda->arguments->children[0]);
const DataTypes & lambda_argument_types = lambda_type->getArgumentTypes();
if (parameters.size() != lambda_argument_types.size())
throw Exception("DataTypeExpression for lambda function has different number of argument types than number of lambda parameters",
ErrorCodes::LOGICAL_ERROR);
for (size_t param_idx = 0, num_params = parameters.size(); param_idx < num_params; ++param_idx)
{
TypeAndConstantInference::ExpressionInfo expression_info;
expression_info.node = typeid_cast<ASTIdentifier *>(lambda->arguments->children[0].get())
? lambda->arguments->children[0]
: typeid_cast<ASTFunction &>(*lambda->arguments->children[0]).arguments->children.at(param_idx);
expression_info.data_type = lambda_argument_types[param_idx];
info.emplace(parameters[param_idx], std::move(expression_info));
}
/// Now dive into.
processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas, table_functions);
/// Update Expression type (expression signature).
info.at(lambda->getColumnName()).data_type = std::make_shared<DataTypeFunction>(
lambda_argument_types, info.at(lambda->arguments->children[1]->getColumnName()).data_type);
}
}
}
void processImpl(
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
const AnalyzeLambdas & lambdas,
ExecuteTableFunctions & table_functions)
{
const ASTFunction * function = typeid_cast<const ASTFunction *>(ast.get());
/// Bottom-up.
/// Don't go into components of compound identifiers.
if (!typeid_cast<const ASTIdentifier *>(ast.get()))
{
for (auto & child : ast->children)
{
/** Don't go into subqueries and table-like expressions.
* Also don't go into components of compound identifiers.
*/
if (typeid_cast<const ASTSelectQuery *>(child.get())
|| typeid_cast<const ASTTableExpression *>(child.get()))
continue;
/** Postpone diving into lambda expressions.
* We must first infer types of other arguments of higher-order function,
* and then process lambda expression.
* Example:
* arrayMap((x, y) -> x + y, arr1, arr2)
* First, infer type of 'arr1' and 'arr2'.
* Then, ask function arrayMap, what types will have 'x' and 'y'.
* And then, infer type of 'x + y'.
*/
if (function && function->name == "lambda")
continue;
processImpl(child, context, aliases, columns, info, lambdas, table_functions);
}
}
const ASTLiteral * literal = nullptr;
const ASTIdentifier * identifier = nullptr;
const ASTSubquery * subquery = nullptr;
function
|| (literal = typeid_cast<const ASTLiteral *>(ast.get()))
|| (identifier = typeid_cast<const ASTIdentifier *>(ast.get()))
|| (subquery = typeid_cast<const ASTSubquery *>(ast.get()));
if (!literal && !identifier && !function && !subquery)
return;
/// Same expression is already processed.
String column_name = ast->getColumnName();
if (info.count(column_name))
return;
if (function)
{
/// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas.
if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast))
processHigherOrderFunction(ast, context, aliases, columns, info, lambdas, table_functions);
processFunction(column_name, ast, info, context);
}
else if (literal)
processLiteral(column_name, ast, info);
else if (identifier)
processIdentifier(column_name, ast, info, context, aliases, columns, lambdas, table_functions);
else if (subquery)
processScalarSubquery(column_name, ast, info, context, table_functions);
}
}
void TypeAndConstantInference::process(ASTPtr & ast, const Context & context,
CollectAliases & aliases,
const AnalyzeColumns & columns,
const AnalyzeLambdas & lambdas,
ExecuteTableFunctions & table_functions)
{
processImpl(ast, context, aliases, columns, info, lambdas, table_functions);
}
void TypeAndConstantInference::dump(WriteBuffer & out) const
{
/// For need of tests, we need to dump result in some fixed order.
std::vector<Info::const_iterator> vec;
vec.reserve(info.size());
for (auto it = info.begin(); it != info.end(); ++it)
vec.emplace_back(it);
std::sort(vec.begin(), vec.end(), [](const auto & a, const auto & b) { return a->first < b->first; });
for (const auto & it : vec)
{
writeString(it->first, out);
writeCString(" -> ", out);
writeString(it->second.data_type->getName(), out);
if (it->second.is_constant_expression)
{
writeCString(" = ", out);
String value = applyVisitor(FieldVisitorToString(), it->second.value);
writeString(value, out);
}
writeCString(". AST: ", out);
if (!it->second.node)
writeCString("(none)", out);
else
{
std::stringstream formatted_ast;
formatAST(*it->second.node, formatted_ast, false, true);
writeString(formatted_ast.str(), out);
}
writeChar('\n', out);
}
}
}

View File

@ -1,61 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <DataTypes/IDataType.h>
#include <Common/UInt128.h>
#include <unordered_map>
namespace DB
{
class Context;
class WriteBuffer;
struct CollectAliases;
struct AnalyzeColumns;
struct AnalyzeLambdas;
struct ExecuteTableFunctions;
class IFunctionBase;
class IAggregateFunction;
/** For every expression, deduce its type,
* and if it is a constant expression, calculate its value.
*
* Types and constants inference goes together,
* because sometimes resulting type of a function depend on value of constant expression.
* Notable examples: tupleElement(tuple, N) and toFixedString(s, N) functions.
*
* Also creates and stores function objects.
* TODO (?) Also calculate ids for expressions, that will identify common subexpressions.
*/
struct TypeAndConstantInference
{
void process(ASTPtr & ast, const Context & context,
CollectAliases & aliases,
const AnalyzeColumns & columns,
const AnalyzeLambdas & analyze_lambdas,
ExecuteTableFunctions & table_functions);
struct ExpressionInfo
{
/// Must identify identical expressions.
/// For example following three expressions in query are the same: SELECT sum(x) AS a, SUM(t.x) AS b, a FROM t
UInt128 id {};
ASTPtr node;
DataTypePtr data_type;
bool is_constant_expression = false;
Field value; /// Has meaning if is_constant_expression == true.
std::shared_ptr<IFunctionBase> function;
std::shared_ptr<IAggregateFunction> aggregate_function;
};
/// Key is getColumnName of AST node.
using Info = std::unordered_map<String, ExpressionInfo>;
Info info;
/// Debug output
void dump(WriteBuffer & out) const;
};
}

View File

@ -1,24 +0,0 @@
add_executable(collect_aliases collect_aliases.cpp)
target_link_libraries(collect_aliases dbms)
add_executable(collect_tables collect_tables.cpp)
target_link_libraries(collect_tables dbms clickhouse_storages_system)
add_executable(analyze_columns analyze_columns.cpp)
target_link_libraries(analyze_columns dbms clickhouse_storages_system)
add_executable(type_and_constant_inference type_and_constant_inference.cpp)
target_link_libraries(type_and_constant_inference
clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions)
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
add_executable(translate_positional_arguments translate_positional_arguments.cpp)
target_link_libraries(translate_positional_arguments dbms)
add_executable(optimize_group_order_limit_by optimize_group_order_limit_by.cpp)
target_link_libraries(optimize_group_order_limit_by dbms clickhouse_storages_system)
add_executable(analyze_lambdas analyze_lambdas.cpp)
target_link_libraries(analyze_lambdas dbms)

View File

@ -1,71 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Analyzers/CollectTables.h>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/AnalyzeLambdas.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Databases/DatabaseMemory.h>
/// Parses query from stdin and print found columns and corresponding tables.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
analyze_lambdas.process(ast);
CollectAliases collect_aliases;
collect_aliases.process(ast);
ExecuteTableFunctions execute_table_functions;
execute_table_functions.process(ast, context);
CollectTables collect_tables;
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
AnalyzeColumns analyze_columns;
analyze_columns.process(ast, collect_aliases, collect_tables);
analyze_columns.dump(out);
out.next();
std::cout << "\n";
formatAST(*ast, std::cout, false, true);
std::cout << "\n";
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,18 +0,0 @@
dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: dummy
number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: number
numbers.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: numbers.number
one.dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: one.dummy
system.numbers.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: system.numbers.number
system.one.dummy -> dummy UInt8. Database name: system. Table name: one. Alias: (none). Storage: SystemOne. AST: system.one.dummy
t.number -> number UInt64. Database name: system. Table name: numbers. Alias: t. Storage: SystemNumbers. AST: t.number
SELECT dummy, number, one.dummy, numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.one.dummy, system.numbers.number, system.numbers.number, t.number FROM system.one , system.numbers AS t
c -> c UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: c
SELECT arrayMap((_lambda0_arg0, _lambda0_arg1) -> arrayMap((_lambda1_arg0, _lambda1_arg1) -> _lambda0_arg0[_lambda1_arg0], _lambda0_arg0, c), [[1], [2, 3]]) FROM (SELECT 1 AS c, 2 AS d)
c -> c UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: c
x -> x UInt8. Database name: (none). Table name: (none). Alias: (none). Storage: (none). AST: x
SELECT x, arrayMap((_lambda0_arg0, _lambda0_arg1) -> (_lambda0_arg0 + _lambda0_arg1), x, c) FROM (SELECT 1 AS x, 2 AS c)

View File

@ -1,7 +0,0 @@
#!/bin/sh
echo "SELECT dummy, number, one.dummy, numbers.number, system.one.dummy, system.numbers.number, one.*, numbers.*, system.one.*, system.numbers.*, *, t.*, t.number FROM system.one, system.numbers AS t" | ./analyze_columns
echo
echo "SELECT arrayMap((x, y) -> arrayMap((y, z) -> x[y], x, c), [[1], [2, 3]]) FROM (SELECT 1 AS c, 2 AS d)" | ./analyze_columns
echo
echo "SELECT x, arrayMap((x, y) -> x + y, x, c) FROM (SELECT 1 AS x, 2 AS c)" | ./analyze_columns

View File

@ -1,44 +0,0 @@
#include <Analyzers/AnalyzeLambdas.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/ParserSelectQuery.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
/// Parses query from stdin and print found higher order functions and query with rewritten names of lambda parameters.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
AnalyzeLambdas analyzer;
analyzer.process(ast);
analyzer.dump(out);
out.next();
std::cout << "\n";
formatAST(*ast, std::cout, false, true);
std::cout << "\n";
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,4 +0,0 @@
arrayMap(lambda(tuple(_lambda1_arg0), arrayElement(_lambda1_arg0, _lambda0_arg1)), arr3)
arrayMap(lambda(tuple(_lambda0_arg0, _lambda0_arg1), plus(_lambda0_arg0, arrayMap(lambda(tuple(_lambda1_arg0), arrayElement(_lambda1_arg0, _lambda0_arg1)), arr3))), arr1, arr2)
SELECT arrayMap((_lambda0_arg0, _lambda0_arg1) -> (_lambda0_arg0 + arrayMap(_lambda1_arg0 -> _lambda1_arg0[_lambda0_arg1], arr3)), arr1, arr2)

View File

@ -1,3 +0,0 @@
#!/bin/sh
echo "SELECT arrayMap((x, y) -> x + arrayMap(x -> x[y], arr3), arr1, arr2)" | ./analyze_lambdas

View File

@ -1,53 +0,0 @@
#include <Analyzers/AnalyzeResultOfQuery.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Databases/DatabaseMemory.h>
/// Parses query from stdin and print names and types of result columns.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
context.setCurrentDatabase("system");
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
ExecuteTableFunctions execute_table_functions;
execute_table_functions.process(ast, context);
AnalyzeResultOfQuery analyzer;
analyzer.process(ast, context, execute_table_functions);
analyzer.dump(out);
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1 +0,0 @@
1 UInt8 ColumnConst<UInt8> 1, plus(2, 3) UInt16 ColumnConst<UInt16> 1, x FixedString(20) ColumnConst<String> 1, y String ColumnConst<String> 1, z Int16 nullptr

View File

@ -1,3 +0,0 @@
#!/bin/sh
echo "SELECT 1, 2 + 3, toFixedString('Hello, world', 20) AS x, ('Hello', 1).1 AS y, z FROM system.numbers WHERE (arrayJoin([-1, 1]) AS z) = 1" | ./analyze_result_of_query

View File

@ -1,36 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
/// Parses query from stdin and print found aliases.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
CollectAliases collector;
collector.process(ast);
collector.dump(out);
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,8 +0,0 @@
a -> (array join) arr AS a
b -> (array join) (arr1 AS z) + 1 AS b
ghi -> (expression) def AS ghi
t -> (table) d.t AS t
u -> (table) (SELECT 1 AS xxx) AS u
x -> (expression) 1 AS x
y -> (expression) x + 2 AS y
z -> (expression) arr1 AS z

View File

@ -1,3 +0,0 @@
#!/bin/sh
echo "SELECT 1 AS x, x + 2 AS y FROM d.t AS t INNER JOIN (SELECT 1 AS xxx) AS u USING (abc, def AS ghi) ARRAY JOIN arr AS a, (arr1 AS z) + 1 AS b" | ./collect_aliases

View File

@ -1,56 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Analyzers/CollectTables.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Databases/DatabaseMemory.h>
/// Parses query from stdin and print found tables.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
context.setCurrentDatabase("system");
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
CollectAliases collect_aliases;
collect_aliases.process(ast);
ExecuteTableFunctions execute_table_functions;
execute_table_functions.process(ast, context);
CollectTables collect_tables;
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
collect_tables.dump(out);
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,80 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Analyzers/CollectTables.h>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/AnalyzeLambdas.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Analyzers/TypeAndConstantInference.h>
#include <Analyzers/TranslatePositionalArguments.h>
#include <Analyzers/OptimizeGroupOrderLimitBy.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Databases/DatabaseMemory.h>
/// Parses query from stdin and print same query with optimized GROUP BY, ORDER BY, LIMIT BY.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
analyze_lambdas.process(ast);
CollectAliases collect_aliases;
collect_aliases.process(ast);
ExecuteTableFunctions execute_table_functions;
execute_table_functions.process(ast, context);
CollectTables collect_tables;
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
AnalyzeColumns analyze_columns;
analyze_columns.process(ast, collect_aliases, collect_tables);
TypeAndConstantInference inference;
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
TranslatePositionalArguments translation;
translation.process(ast);
OptimizeGroupOrderLimitBy optimizer;
optimizer.process(ast, inference);
formatAST(*ast, std::cout, false);
std::cout << "\n";
return 0;
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,19 +0,0 @@
SELECT
number,
materialize('abc')
FROM
(
SELECT
number,
10 AS b
FROM system.numbers
)
GROUP BY number
SELECT number
FROM system.numbers
GROUP BY number
ORDER BY
number ASC,
rand() ASC,
toString(rand()) ASC COLLATE 'tr'

View File

@ -1,5 +0,0 @@
#!/bin/sh
echo "SELECT number, materialize('abc') FROM (SELECT number, 10 AS b FROM system.numbers) GROUP BY number, toString(number + 1), number + number, 1, 2, 'Hello', b" | ./optimize_group_order_limit_by
echo
echo "SELECT number FROM system.numbers GROUP BY 1 ORDER BY number, 'hello' DESC COLLATE 'tr', number + 1, rand(), identity(number * 2, rand()), toString(rand()) COLLATE 'tr'" | ./optimize_group_order_limit_by

View File

@ -1,37 +0,0 @@
#include <Analyzers/TranslatePositionalArguments.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
/// Parses query from stdin and print same query with translated positional arguments.
int main(int, char **)
try
{
using namespace DB;
ReadBufferFromFileDescriptor in(STDIN_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
TranslatePositionalArguments translator;
translator.process(ast);
formatAST(*ast, std::cout, false);
std::cout << "\n";
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,8 +0,0 @@
SELECT
abc,
def + 1,
count()
GROUP BY
abc,
def + 1
ORDER BY abc DESC

View File

@ -1,3 +0,0 @@
#!/bin/sh
echo "SELECT abc, def + 1, count() GROUP BY 1, 2 ORDER BY 1 DESC" | ./translate_positional_arguments

View File

@ -1,78 +0,0 @@
#include <Analyzers/CollectAliases.h>
#include <Analyzers/CollectTables.h>
#include <Analyzers/AnalyzeColumns.h>
#include <Analyzers/AnalyzeLambdas.h>
#include <Analyzers/ExecuteTableFunctions.h>
#include <Analyzers/TypeAndConstantInference.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Databases/DatabaseMemory.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values.
int main(int, char **)
try
{
using namespace DB;
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
String query;
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
analyze_lambdas.process(ast);
CollectAliases collect_aliases;
collect_aliases.process(ast);
ExecuteTableFunctions execute_table_functions;
execute_table_functions.process(ast, context);
CollectTables collect_tables;
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
AnalyzeColumns analyze_columns;
analyze_columns.process(ast, collect_aliases, collect_tables);
TypeAndConstantInference inference;
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
inference.dump(out);
out.next();
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
return 1;
}

View File

@ -1,34 +0,0 @@
' ' -> String = ' '. AST: ' '
'Hello' -> String = 'Hello'. AST: 'Hello'
'Hello, world' -> String = 'Hello, world'. AST: 'Hello, world'
'World' -> String = 'World'. AST: 'World'
0.5 -> Float64 = 0.5. AST: 0.5
1 -> UInt8 = 1. AST: 1
20 -> UInt8 = 20. AST: 20
[1, -1, 0.1] -> Array(Float64) = [1., -1., 0.1]. AST: [1, -1, 0.1]
concat('Hello', ' ', 'World') -> String = 'Hello World'. AST: concat('Hello', ' ', 'World') AS world
count() -> UInt64. AST: count()
e -> Float64 = 2.718281828459045. AST: e() AS e
e() -> Float64 = 2.718281828459045. AST: e() AS e
pi() -> Float64 = 3.141592653589793. AST: pi()
plus(e, pi()) -> Float64 = 5.859874482048838. AST: e + pi()
quantileTiming(0.5)(1) -> Float32. AST: quantileTiming(0.5)(1)
rand() -> UInt32. AST: rand()
sleep(1) -> UInt8. AST: sleep(1)
sum(e) -> Float64. AST: sum(e)
system.numbers.number -> UInt64. AST: system.numbers.number
toFixedString('Hello, world', 20) -> FixedString(20) = 'Hello, world\0\0\0\0\0\0\0\0'. AST: toFixedString('Hello, world', 20)
tuple('Hello', 1) -> Tuple(String, UInt8) = ('Hello', 1). AST: ('Hello', 1)
tupleElement(tuple('Hello', 1), 1) -> String = 'Hello'. AST: ('Hello', 1).1
t.x -> UInt8 = 1. AST: t.x
x -> UInt8 = 1. AST: x
1 -> UInt8 = 1. AST: 1
t.x -> UInt8 = 1. AST: t.x
x -> UInt8 = 1. AST: x
x -> Tuple(UInt8, UInt8) = (1, 2). AST: x
y -> Tuple(UInt8, UInt8) = (3, 4). AST: y
z -> String = 'Hello, world'. AST: z

View File

@ -1,26 +0,0 @@
#!/bin/sh
echo "SELECT
1,
*,
sleep(1),
rand(),
'Hello' || ' ' || 'World' AS world,
e() AS e,
e + pi(),
count(*),
sum(e),
quantileTiming(0.5)(1),
[1, -1, 0.1],
toFixedString('Hello, world', 20),
('Hello', 1).1
FROM system.numbers" | ./type_and_constant_inference
echo
echo "SELECT t.x FROM (SELECT 1 AS x) AS t" | ./type_and_constant_inference
echo
echo "SELECT x FROM (SELECT 1 AS x)" | ./type_and_constant_inference
echo
echo "SELECT t.x, x, 1 FROM (SELECT 1 AS x) AS t" | ./type_and_constant_inference
echo
echo "SELECT *, z FROM (SELECT (1, 2) AS x, (SELECT 3, 4) AS y), (SELECT 'Hello, world' AS z)" | ./type_and_constant_inference

View File

@ -12,5 +12,4 @@ add_subdirectory (Interpreters)
add_subdirectory (AggregateFunctions)
add_subdirectory (Client)
add_subdirectory (TableFunctions)
add_subdirectory (Analyzers)
add_subdirectory (Formats)

View File

@ -31,6 +31,7 @@ public:
*/
using Base = COWPtrHelper<IColumn, ColumnTuple>;
static Ptr create(const Columns & columns);
static Ptr create(Columns && arg) { return create(arg); }
template <typename Arg, typename = typename std::enable_if<std::is_rvalue_reference<Arg &&>::value>::type>
static MutablePtr create(Arg && arg) { return Base::create(std::forward<Arg>(arg)); }

View File

@ -1,19 +1,16 @@
#include "ColumnVector.h"
#include <cstring>
#include <cmath>
#include <common/unaligned.h>
#include <Common/Exception.h>
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/NaNUtils.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsCommon.h>
#include <DataStreams/ColumnGathererStream.h>
#include <ext/bit_cast.h>
#if __SSE2__

View File

@ -23,7 +23,7 @@ public:
/// Uses thread-safe cache.
virtual const UInt64 * tryGetSavedHash() const = 0;
size_t size() const override { return getNestedColumn()->size(); }
size_t size() const override { return getNestedNotNullableColumn()->size(); }
/// Appends new value at the end of column (column's size is increased by 1).
/// Is used to transform raw strings to Blocks (for example, inside input format parsers)

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
}
void throwFromErrno(const std::string & s, int code, int e)
std::string errnoToString(int code, int e)
{
const size_t buf_size = 128;
char buf[buf_size];
@ -43,12 +43,17 @@ void throwFromErrno(const std::string & s, int code, int e)
strcpy(buf, unknown_message);
strcpy(buf + strlen(unknown_message), code);
}
throw ErrnoException(s + ", errno: " + toString(e) + ", strerror: " + std::string(buf), code, e);
return "errno: " + toString(e) + ", strerror: " + std::string(buf);
#else
throw ErrnoException(s + ", errno: " + toString(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf))), code, e);
(void)code;
return "errno: " + toString(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf)));
#endif
}
void throwFromErrno(const std::string & s, int code, int e)
{
throw ErrnoException(s + ", " + errnoToString(code, e));
}
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
{

View File

@ -59,6 +59,7 @@ private:
using Exceptions = std::vector<std::exception_ptr>;
std::string errnoToString(int code = 0, int the_errno = errno);
[[noreturn]] void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno);

View File

@ -39,7 +39,7 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
#include <nmmintrin.h>
#endif
#if __aarch64__
#if __aarch64__ && __ARM_FEATURE_CRC32
#include <arm_acle.h>
#include <arm_neon.h>
#endif
@ -48,8 +48,8 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x)
{
#if __SSE4_2__
return _mm_crc32_u64(-1ULL, x);
#elif __aarch64__
return __crc32cd(-1ULL, x);
#elif __aarch64__ && __ARM_FEATURE_CRC32
return __crc32cd(-1U, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x);

View File

@ -494,12 +494,35 @@ public:
alloc(grower);
}
HashTable(HashTable && rhs)
: buf(nullptr)
{
*this = std::move(rhs);
}
~HashTable()
{
destroyElements();
free();
}
HashTable & operator= (HashTable && rhs)
{
destroyElements();
free();
std::swap(buf, rhs.buf);
std::swap(m_size, rhs.m_size);
std::swap(grower, rhs.grower);
Hash::operator=(std::move(rhs));
Allocator::operator=(std::move(rhs));
Cell::State::operator=(std::move(rhs));
ZeroValueStorage<Cell::need_zero_value_storage, Cell>::operator=(std::move(rhs));
return *this;
}
class Reader final : private Cell::State
{
public:

View File

@ -81,15 +81,11 @@ StatusFile::StatusFile(const std::string & path_)
StatusFile::~StatusFile()
{
char buf[128];
if (0 != close(fd))
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", errno: "
<< errno << ", strerror: " << strerror_r(errno, buf, sizeof(buf)));
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString());
if (0 != unlink(path.c_str()))
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", errno: "
<< errno << ", strerror: " << strerror_r(errno, buf, sizeof(buf)));
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString());
}
}

View File

@ -1,4 +1,4 @@
#include <Common/TaskStatsInfoGetter.h>
#include "TaskStatsInfoGetter.h"
#include <Common/Exception.h>
#include <Core/Types.h>
@ -6,6 +6,7 @@
#if defined(__linux__)
#include "hasLinuxCapability.h"
#include <common/unaligned.h>
#include <errno.h>
@ -185,18 +186,8 @@ UInt16 getFamilyIdImpl(int fd)
bool checkPermissionsImpl()
{
/// See man getcap.
__user_cap_header_struct request{};
request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested.
request.pid = getpid();
__user_cap_data_struct response{};
/// Avoid dependency on 'libcap'.
if (0 != syscall(SYS_capget, &request, &response))
throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR);
if (!((1 << CAP_NET_ADMIN) & response.effective))
static bool res = hasLinuxCapability(CAP_NET_ADMIN);
if (!res)
return false;
/// Check that we can successfully initialize TaskStatsInfoGetter.

View File

@ -4,6 +4,7 @@
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Interpreters/Context.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h>

View File

@ -0,0 +1,44 @@
#if defined(__linux__)
#include "hasLinuxCapability.h"
#include <syscall.h>
#include <unistd.h>
#include <linux/capability.h>
#include <linux/netlink.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NETLINK_ERROR;
}
static __user_cap_data_struct getCapabilities()
{
/// See man getcap.
__user_cap_header_struct request{};
request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested.
request.pid = getpid();
__user_cap_data_struct response{};
/// Avoid dependency on 'libcap'.
if (0 != syscall(SYS_capget, &request, &response))
throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR);
return response;
}
bool hasLinuxCapability(int cap)
{
static __user_cap_data_struct capabilities = getCapabilities();
return (1 << cap) & capabilities.effective;
}
}
#endif

View File

@ -0,0 +1,13 @@
#if defined(__linux__)
#include <linux/capability.h>
namespace DB
{
/// Check that the current process has Linux capability. Examples: CAP_IPC_LOCK, CAP_NET_ADMIN.
bool hasLinuxCapability(int cap);
}
#endif

View File

@ -8,6 +8,7 @@
#include <Common/ArenaWithFreeLists.h>
#endif
#include <variant>
#include <memory>
#include <array>
#include <sys/resource.h>
@ -144,12 +145,12 @@ struct Dictionary
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<
std::variant<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,

View File

@ -16,6 +16,7 @@
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
@ -35,7 +36,9 @@ try
Context context = Context::createGlobal();
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
ExpressionAnalyzer analyzer(ast, syntax_result, context);
ExpressionActionsChain chain(context);
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain);

View File

@ -18,6 +18,7 @@
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
@ -40,7 +41,9 @@ try
Context context = Context::createGlobal();
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
ExpressionAnalyzer analyzer(ast, syntax_result, context);
ExpressionActionsChain chain(context);
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain);

View File

@ -454,79 +454,30 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib
switch (type)
{
case AttributeUnderlyingType::UInt8:
std::get<UInt8>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt8>>(attr.arrays) = std::make_unique<ContainerType<UInt8>>(size);
bytes_allocated += size * sizeof(UInt8);
break;
case AttributeUnderlyingType::UInt16:
std::get<UInt16>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt16>>(attr.arrays) = std::make_unique<ContainerType<UInt16>>(size);
bytes_allocated += size * sizeof(UInt16);
break;
case AttributeUnderlyingType::UInt32:
std::get<UInt32>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt32>>(attr.arrays) = std::make_unique<ContainerType<UInt32>>(size);
bytes_allocated += size * sizeof(UInt32);
break;
case AttributeUnderlyingType::UInt64:
std::get<UInt64>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt64>>(attr.arrays) = std::make_unique<ContainerType<UInt64>>(size);
bytes_allocated += size * sizeof(UInt64);
break;
case AttributeUnderlyingType::UInt128:
std::get<UInt128>(attr.null_values) = null_value.get<UInt128>();
std::get<ContainerPtrType<UInt128>>(attr.arrays) = std::make_unique<ContainerType<UInt128>>(size);
bytes_allocated += size * sizeof(UInt128);
break;
case AttributeUnderlyingType::Int8:
std::get<Int8>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int8>>(attr.arrays) = std::make_unique<ContainerType<Int8>>(size);
bytes_allocated += size * sizeof(Int8);
break;
case AttributeUnderlyingType::Int16:
std::get<Int16>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int16>>(attr.arrays) = std::make_unique<ContainerType<Int16>>(size);
bytes_allocated += size * sizeof(Int16);
break;
case AttributeUnderlyingType::Int32:
std::get<Int32>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int32>>(attr.arrays) = std::make_unique<ContainerType<Int32>>(size);
bytes_allocated += size * sizeof(Int32);
break;
case AttributeUnderlyingType::Int64:
std::get<Int64>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int64>>(attr.arrays) = std::make_unique<ContainerType<Int64>>(size);
bytes_allocated += size * sizeof(Int64);
break;
case AttributeUnderlyingType::Decimal32:
std::get<Decimal32>(attr.null_values) = null_value.get<Decimal32>();
std::get<ContainerPtrType<Decimal32>>(attr.arrays) = std::make_unique<ContainerType<Decimal32>>(size);
bytes_allocated += size * sizeof(Decimal32);
break;
case AttributeUnderlyingType::Decimal64:
std::get<Decimal64>(attr.null_values) = null_value.get<Decimal64>();
std::get<ContainerPtrType<Decimal64>>(attr.arrays) = std::make_unique<ContainerType<Decimal64>>(size);
bytes_allocated += size * sizeof(Decimal64);
break;
case AttributeUnderlyingType::Decimal128:
std::get<Decimal128>(attr.null_values) = null_value.get<Decimal128>();
std::get<ContainerPtrType<Decimal128>>(attr.arrays) = std::make_unique<ContainerType<Decimal128>>(size);
bytes_allocated += size * sizeof(Decimal128);
break;
case AttributeUnderlyingType::Float32:
std::get<Float32>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float32>>(attr.arrays) = std::make_unique<ContainerType<Float32>>(size);
bytes_allocated += size * sizeof(Float32);
break;
case AttributeUnderlyingType::Float64:
std::get<Float64>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float64>>(attr.arrays) = std::make_unique<ContainerType<Float64>>(size);
bytes_allocated += size * sizeof(Float64);
#define DISPATCH(TYPE) \
case AttributeUnderlyingType::TYPE: \
attr.null_values = TYPE(null_value.get<typename NearestFieldType<TYPE>::Type>()); \
attr.arrays = std::make_unique<ContainerType<TYPE>>(size); \
bytes_allocated += size * sizeof(TYPE); \
break;
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
DISPATCH(UInt64)
DISPATCH(UInt128)
DISPATCH(Int8)
DISPATCH(Int16)
DISPATCH(Int32)
DISPATCH(Int64)
DISPATCH(Decimal32)
DISPATCH(Decimal64)
DISPATCH(Decimal128)
DISPATCH(Float32)
DISPATCH(Float64)
#undef DISPATCH
case AttributeUnderlyingType::String:
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.arrays) = std::make_unique<ContainerType<StringRef>>(size);
attr.null_values = null_value.get<String>();
attr.arrays = std::make_unique<ContainerType<StringRef>>(size);
bytes_allocated += size * sizeof(StringRef);
if (!string_arena)
string_arena = std::make_unique<ArenaWithFreeLists>();

View File

@ -13,7 +13,7 @@
#include <chrono>
#include <vector>
#include <map>
#include <tuple>
#include <variant>
#include <pcg_random.hpp>
#include <shared_mutex>
@ -182,14 +182,14 @@ private:
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::tuple<
std::variant<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,

View File

@ -3,7 +3,7 @@
#include <atomic>
#include <chrono>
#include <map>
#include <tuple>
#include <variant>
#include <vector>
#include <shared_mutex>
#include <Columns/ColumnDecimal.h>
@ -216,12 +216,9 @@ public:
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value>
using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value>
using ContainerType = Value[];
template <typename Value>
using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
template <typename Value> using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value> using ContainerType = Value[];
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct CellMetadata final
{
@ -260,10 +257,10 @@ private:
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64,
std::variant<UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64, String> null_values;
std::tuple<ContainerPtrType<UInt8>,
std::variant<ContainerPtrType<UInt8>,
ContainerPtrType<UInt16>,
ContainerPtrType<UInt32>,
ContainerPtrType<UInt64>,

View File

@ -9,79 +9,30 @@ ComplexKeyCacheDictionary::Attribute ComplexKeyCacheDictionary::createAttributeW
switch (type)
{
case AttributeUnderlyingType::UInt8:
std::get<UInt8>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt8>>(attr.arrays) = std::make_unique<ContainerType<UInt8>>(size);
bytes_allocated += size * sizeof(UInt8);
break;
case AttributeUnderlyingType::UInt16:
std::get<UInt16>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt16>>(attr.arrays) = std::make_unique<ContainerType<UInt16>>(size);
bytes_allocated += size * sizeof(UInt16);
break;
case AttributeUnderlyingType::UInt32:
std::get<UInt32>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt32>>(attr.arrays) = std::make_unique<ContainerType<UInt32>>(size);
bytes_allocated += size * sizeof(UInt32);
break;
case AttributeUnderlyingType::UInt64:
std::get<UInt64>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt64>>(attr.arrays) = std::make_unique<ContainerType<UInt64>>(size);
bytes_allocated += size * sizeof(UInt64);
break;
case AttributeUnderlyingType::UInt128:
std::get<UInt128>(attr.null_values) = null_value.get<UInt128>();
std::get<ContainerPtrType<UInt128>>(attr.arrays) = std::make_unique<ContainerType<UInt128>>(size);
bytes_allocated += size * sizeof(UInt128);
break;
case AttributeUnderlyingType::Int8:
std::get<Int8>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int8>>(attr.arrays) = std::make_unique<ContainerType<Int8>>(size);
bytes_allocated += size * sizeof(Int8);
break;
case AttributeUnderlyingType::Int16:
std::get<Int16>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int16>>(attr.arrays) = std::make_unique<ContainerType<Int16>>(size);
bytes_allocated += size * sizeof(Int16);
break;
case AttributeUnderlyingType::Int32:
std::get<Int32>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int32>>(attr.arrays) = std::make_unique<ContainerType<Int32>>(size);
bytes_allocated += size * sizeof(Int32);
break;
case AttributeUnderlyingType::Int64:
std::get<Int64>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int64>>(attr.arrays) = std::make_unique<ContainerType<Int64>>(size);
bytes_allocated += size * sizeof(Int64);
break;
case AttributeUnderlyingType::Float32:
std::get<Float32>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float32>>(attr.arrays) = std::make_unique<ContainerType<Float32>>(size);
bytes_allocated += size * sizeof(Float32);
break;
case AttributeUnderlyingType::Float64:
std::get<Float64>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float64>>(attr.arrays) = std::make_unique<ContainerType<Float64>>(size);
bytes_allocated += size * sizeof(Float64);
break;
case AttributeUnderlyingType::Decimal32:
std::get<Decimal32>(attr.null_values) = null_value.get<Decimal32>();
std::get<ContainerPtrType<Decimal32>>(attr.arrays) = std::make_unique<ContainerType<Decimal32>>(size);
bytes_allocated += size * sizeof(Decimal32);
break;
case AttributeUnderlyingType::Decimal64:
std::get<Decimal64>(attr.null_values) = null_value.get<Decimal64>();
std::get<ContainerPtrType<Decimal64>>(attr.arrays) = std::make_unique<ContainerType<Decimal64>>(size);
bytes_allocated += size * sizeof(Decimal64);
break;
case AttributeUnderlyingType::Decimal128:
std::get<Decimal128>(attr.null_values) = null_value.get<Decimal128>();
std::get<ContainerPtrType<Decimal128>>(attr.arrays) = std::make_unique<ContainerType<Decimal128>>(size);
bytes_allocated += size * sizeof(Decimal128);
#define DISPATCH(TYPE) \
case AttributeUnderlyingType::TYPE: \
attr.null_values = TYPE(null_value.get<typename NearestFieldType<TYPE>::Type>()); \
attr.arrays = std::make_unique<ContainerType<TYPE>>(size); \
bytes_allocated += size * sizeof(TYPE); \
break;
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
DISPATCH(UInt64)
DISPATCH(UInt128)
DISPATCH(Int8)
DISPATCH(Int16)
DISPATCH(Int32)
DISPATCH(Int64)
DISPATCH(Decimal32)
DISPATCH(Decimal64)
DISPATCH(Decimal128)
DISPATCH(Float32)
DISPATCH(Float64)
#undef DISPATCH
case AttributeUnderlyingType::String:
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.arrays) = std::make_unique<ContainerType<StringRef>>(size);
attr.null_values = null_value.get<String>();
attr.arrays = std::make_unique<ContainerType<StringRef>>(size);
bytes_allocated += size * sizeof(StringRef);
if (!string_arena)
string_arena = std::make_unique<ArenaWithFreeLists>();

View File

@ -376,9 +376,9 @@ void ComplexKeyHashedDictionary::loadData()
template <typename T>
void ComplexKeyHashedDictionary::addAttributeSize(const Attribute & attribute)
{
const auto & map_ref = std::get<ContainerPtrType<T>>(attribute.maps);
bytes_allocated += sizeof(ContainerType<T>) + map_ref->getBufferSizeInBytes();
bucket_count = map_ref->getBufferSizeInCells();
const auto & map_ref = std::get<ContainerType<T>>(attribute.maps);
bytes_allocated += sizeof(ContainerType<T>) + map_ref.getBufferSizeInBytes();
bucket_count = map_ref.getBufferSizeInCells();
}
void ComplexKeyHashedDictionary::calculateBytesAllocated()
@ -421,8 +421,8 @@ void ComplexKeyHashedDictionary::calculateBytesAllocated()
template <typename T>
void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<ContainerPtrType<T>>(attribute.maps) = std::make_unique<ContainerType<T>>();
attribute.null_values = T(null_value.get<typename NearestFieldType<T>::Type>());
attribute.maps.emplace<ContainerType<T>>();
}
ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -449,8 +449,8 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.maps) = std::make_unique<ContainerType<StringRef>>();
attr.null_values = null_value.get<String>();
attr.maps.emplace<ContainerType<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
@ -497,7 +497,7 @@ void ComplexKeyHashedDictionary::getItemsImpl(
ValueSetter && set_value,
DefaultGetter && get_default) const
{
const auto & attr = *std::get<ContainerPtrType<AttributeType>>(attribute.maps);
const auto & attr = std::get<ContainerType<AttributeType>>(attribute.maps);
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
@ -523,7 +523,7 @@ void ComplexKeyHashedDictionary::getItemsImpl(
template <typename T>
bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value)
{
auto & map = *std::get<ContainerPtrType<T>>(attribute.maps);
auto & map = std::get<ContainerType<T>>(attribute.maps);
const auto pair = map.insert({ key, value });
return pair.second;
}
@ -550,7 +550,7 @@ bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const
case AttributeUnderlyingType::String:
{
auto & map = *std::get<ContainerPtrType<StringRef>>(attribute.maps);
auto & map = std::get<ContainerType<StringRef>>(attribute.maps);
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
const auto pair = map.insert({ key, StringRef{string_in_arena, string.size()} });
@ -596,7 +596,7 @@ StringRef ComplexKeyHashedDictionary::placeKeysInPool(
template <typename T>
void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const
{
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto & attr = std::get<ContainerType<T>>(attribute.maps);
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
@ -646,7 +646,7 @@ std::vector<StringRef> ComplexKeyHashedDictionary::getKeys() const
template <typename T>
std::vector<StringRef> ComplexKeyHashedDictionary::getKeys(const Attribute & attribute) const
{
const ContainerType<T> & attr = *std::get<ContainerPtrType<T>>(attribute.maps);
const ContainerType<T> & attr = std::get<ContainerType<T>>(attribute.maps);
std::vector<StringRef> keys;
keys.reserve(attr.size());
for (const auto & key : attr)

View File

@ -11,7 +11,7 @@
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <tuple>
#include <variant>
namespace DB
@ -147,25 +147,24 @@ public:
private:
template <typename Value> using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>, ContainerPtrType<Decimal64>, ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> maps;
std::variant<
ContainerType<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> maps;
std::unique_ptr<Arena> string_arena;
};

View File

@ -70,7 +70,7 @@ void FlatDictionary::isInImpl(
PaddedPODArray<UInt8> & out) const
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
const auto & attr = *std::get<ContainerPtrType<Key>>(hierarchical_attribute->arrays);
const auto & attr = std::get<ContainerType<Key>>(hierarchical_attribute->arrays);
const auto rows = out.size();
size_t loaded_size = attr.size();
@ -395,9 +395,9 @@ void FlatDictionary::loadData()
template <typename T>
void FlatDictionary::addAttributeSize(const Attribute & attribute)
{
const auto & array_ref = std::get<ContainerPtrType<T>>(attribute.arrays);
bytes_allocated += sizeof(PaddedPODArray<T>) + array_ref->allocated_bytes();
bucket_count = array_ref->capacity();
const auto & array_ref = std::get<ContainerType<T>>(attribute.arrays);
bytes_allocated += sizeof(PaddedPODArray<T>) + array_ref.allocated_bytes();
bucket_count = array_ref.capacity();
}
@ -440,22 +440,19 @@ void FlatDictionary::calculateBytesAllocated()
template <typename T>
void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
const auto & null_value_ref = std::get<T>(attribute.null_values) =
null_value.get<typename NearestFieldType<T>::Type>();
std::get<ContainerPtrType<T>>(attribute.arrays) =
std::make_unique<ContainerType<T>>(initial_array_size, null_value_ref);
attribute.null_values = T(null_value.get<typename NearestFieldType<T>::Type>());
const auto & null_value_ref = std::get<T>(attribute.null_values);
attribute.arrays.emplace<ContainerType<T>>(initial_array_size, null_value_ref);
}
template <>
void FlatDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
{
attribute.string_arena = std::make_unique<Arena>();
auto & null_value_ref = std::get<StringRef>(attribute.null_values);
const String & string = null_value.get<typename NearestFieldType<String>::Type>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
null_value_ref = StringRef{string_in_arena, string.size()};
std::get<ContainerPtrType<StringRef>>(attribute.arrays) =
std::make_unique<ContainerType<StringRef>>(initial_array_size, null_value_ref);
const String & string = null_value.get<String>();
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
attribute.arrays.emplace<ContainerType<StringRef>>(initial_array_size, StringRef(string_in_arena, string.size()));
}
@ -525,7 +522,7 @@ void FlatDictionary::getItemsImpl(
ValueSetter && set_value,
DefaultGetter && get_default) const
{
const auto & attr = *std::get<ContainerPtrType<AttributeType>>(attribute.arrays);
const auto & attr = std::get<ContainerType<AttributeType>>(attribute.arrays);
const auto rows = ext::size(ids);
for (const auto row : ext::range(0, rows))
@ -543,7 +540,7 @@ void FlatDictionary::resize(Attribute & attribute, const Key id)
if (id >= max_array_size)
throw Exception{name + ": identifier should be less than " + toString(max_array_size), ErrorCodes::ARGUMENT_OUT_OF_BOUND};
auto & array = *std::get<ContainerPtrType<T>>(attribute.arrays);
auto & array = std::get<ContainerType<T>>(attribute.arrays);
if (id >= array.size())
{
const size_t elements_count = id + 1; //id=0 -> elements_count=1
@ -556,7 +553,7 @@ template <typename T>
void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T & value)
{
resize<T>(attribute, id);
auto & array = *std::get<ContainerPtrType<T>>(attribute.arrays);
auto & array = std::get<ContainerType<T>>(attribute.arrays);
array[id] = value;
loaded_ids[id] = true;
}
@ -566,7 +563,7 @@ void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, const
{
resize<StringRef>(attribute, id);
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
auto & array = *std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto & array = std::get<ContainerType<StringRef>>(attribute.arrays);
array[id] = StringRef{string_in_arena, string.size()};
loaded_ids[id] = true;
}

View File

@ -10,7 +10,7 @@
#include <ext/size.h>
#include <atomic>
#include <vector>
#include <tuple>
#include <variant>
namespace DB
@ -147,25 +147,24 @@ public:
private:
template <typename Value> using ContainerType = PaddedPODArray<Value>;
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
StringRef> null_values;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>, ContainerPtrType<Decimal64>, ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> arrays;
std::variant<
ContainerType<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> arrays;
std::unique_ptr<Arena> string_arena;
};

View File

@ -430,8 +430,8 @@ void HashedDictionary::calculateBytesAllocated()
template <typename T>
void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<CollectionPtrType<T>>(attribute.maps) = std::make_unique<CollectionType<T>>();
attribute.null_values = T(null_value.get<typename NearestFieldType<T>::Type>());
attribute.maps = std::make_unique<CollectionType<T>>();
}
HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -458,8 +458,8 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<CollectionPtrType<StringRef>>(attr.maps) = std::make_unique<CollectionType<StringRef>>();
attr.null_values = null_value.get<String>();
attr.maps = std::make_unique<CollectionType<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}

View File

@ -9,7 +9,7 @@
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <tuple>
#include <variant>
namespace DB
@ -150,14 +150,14 @@ private:
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::tuple<
std::variant<
CollectionPtrType<UInt8>, CollectionPtrType<UInt16>, CollectionPtrType<UInt32>, CollectionPtrType<UInt64>,
CollectionPtrType<UInt128>,
CollectionPtrType<Int8>, CollectionPtrType<Int16>, CollectionPtrType<Int32>, CollectionPtrType<Int64>,

View File

@ -261,8 +261,8 @@ void RangeHashedDictionary::calculateBytesAllocated()
template <typename T>
void RangeHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<Ptr<T>>(attribute.maps) = std::make_unique<Collection<T>>();
attribute.null_values = T(null_value.get<typename NearestFieldType<T>::Type>());
attribute.maps = std::make_unique<Collection<T>>();
}
RangeHashedDictionary::Attribute RangeHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -289,8 +289,8 @@ RangeHashedDictionary::Attribute RangeHashedDictionary::createAttributeWithType(
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<Ptr<StringRef>>(attr.maps) = std::make_unique<Collection<StringRef>>();
attr.null_values = null_value.get<String>();
attr.maps = std::make_unique<Collection<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}

View File

@ -9,7 +9,7 @@
#include <atomic>
#include <memory>
#include <tuple>
#include <variant>
namespace DB
@ -118,13 +118,13 @@ private:
{
public:
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
std::variant<UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::tuple<Ptr<UInt8>, Ptr<UInt16>, Ptr<UInt32>, Ptr<UInt64>,
std::variant<Ptr<UInt8>, Ptr<UInt16>, Ptr<UInt32>, Ptr<UInt64>,
Ptr<UInt128>,
Ptr<Int8>, Ptr<Int16>, Ptr<Int32>, Ptr<Int64>,
Ptr<Decimal32>, Ptr<Decimal64>, Ptr<Decimal128>,

View File

@ -295,7 +295,7 @@ void TrieDictionary::loadData()
template <typename T>
void TrieDictionary::addAttributeSize(const Attribute & attribute)
{
const auto & vec = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto & vec = std::get<ContainerType<T>>(attribute.maps);
bytes_allocated += sizeof(ContainerType<T>) + (vec.capacity() * sizeof(T));
bucket_count = vec.size();
}
@ -352,8 +352,8 @@ void TrieDictionary::validateKeyTypes(const DataTypes & key_types) const
template <typename T>
void TrieDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<ContainerPtrType<T>>(attribute.maps) = std::make_unique<ContainerType<T>>();
attribute.null_values = T(null_value.get<typename NearestFieldType<T>::Type>());
attribute.maps.emplace<ContainerType<T>>();
}
TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -380,8 +380,8 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.maps) = std::make_unique<ContainerType<StringRef>>();
attr.null_values = null_value.get<String>();
attr.maps.emplace<ContainerType<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
@ -428,7 +428,7 @@ void TrieDictionary::getItemsImpl(
ValueSetter && set_value,
DefaultGetter && get_default) const
{
auto & vec = *std::get<ContainerPtrType<AttributeType>>(attribute.maps);
auto & vec = std::get<ContainerType<AttributeType>>(attribute.maps);
const auto first_column = key_columns.front();
const auto rows = first_column->size();
@ -462,7 +462,7 @@ template <typename T>
bool TrieDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value)
{
// Insert value into appropriate vector type
auto & vec = *std::get<ContainerPtrType<T>>(attribute.maps);
auto & vec = std::get<ContainerType<T>>(attribute.maps);
size_t row = vec.size();
vec.push_back(value);

View File

@ -11,7 +11,7 @@
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <tuple>
#include <variant>
#include <common/logger_useful.h>
struct btrie_s;
@ -150,25 +150,24 @@ public:
private:
template <typename Value> using ContainerType = std::vector<Value>;
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct Attribute final
{
AttributeUnderlyingType type;
std::tuple<
std::variant<
UInt8, UInt16, UInt32, UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>, ContainerPtrType<Decimal64>, ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> maps;
std::variant<
ContainerType<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> maps;
std::unique_ptr<Arena> string_arena;
};

View File

@ -1,3 +1,4 @@
#pragma once
#include <Core/Types.h>
#include <Common/Exception.h>
#include <common/DateLUTImpl.h>

Some files were not shown because too many files have changed in this diff Show More