Merge branch 'master' into watchdog

This commit is contained in:
Alexey Milovidov 2020-12-22 19:54:57 +03:00
commit ea3db22c1a
94 changed files with 975 additions and 897 deletions

3
.gitmodules vendored
View File

@ -53,7 +53,8 @@
url = https://github.com/ClickHouse-Extras/Turbo-Base64.git
[submodule "contrib/arrow"]
path = contrib/arrow
url = https://github.com/apache/arrow
url = https://github.com/ClickHouse-Extras/arrow
branch = clickhouse-arrow-2.0.0
[submodule "contrib/thrift"]
path = contrib/thrift
url = https://github.com/apache/thrift.git

View File

@ -6,6 +6,12 @@
#include <unistd.h>
#include <functional>
#include <sys/file.h>
#include <sys/types.h>
#include <sys/wait.h>
#include <csignal>
#include <dlfcn.h>
#include <fcntl.h>
#include <fstream>
namespace
{
@ -83,6 +89,8 @@ ReplxxLineReader::ReplxxLineReader(
/// it also binded to M-p/M-n).
rx.bind_key(Replxx::KEY::meta('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::COMPLETE_NEXT, code); });
rx.bind_key(Replxx::KEY::meta('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::COMPLETE_PREVIOUS, code); });
rx.bind_key(Replxx::KEY::meta('E'), [this](char32_t) { openEditor(); return Replxx::ACTION_RESULT::CONTINUE; });
}
ReplxxLineReader::~ReplxxLineReader()
@ -127,7 +135,114 @@ void ReplxxLineReader::addToHistory(const String & line)
rx.print("Unlock of history file failed: %s\n", errnoToString(errno).c_str());
}
int ReplxxLineReader::execute(const std::string & command)
{
std::vector<char> argv0("sh", &("sh"[3]));
std::vector<char> argv1("-c", &("-c"[3]));
std::vector<char> argv2(command.data(), command.data() + command.size() + 1);
const char * filename = "/bin/sh";
char * const argv[] = {argv0.data(), argv1.data(), argv2.data(), nullptr};
static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork");
if (!real_vfork)
{
rx.print("Cannot find symbol vfork in myself: %s\n", errnoToString(errno).c_str());
return -1;
}
pid_t pid = reinterpret_cast<pid_t (*)()>(real_vfork)();
if (-1 == pid)
{
rx.print("Cannot vfork: %s\n", errnoToString(errno).c_str());
return -1;
}
if (0 == pid)
{
sigset_t mask;
sigemptyset(&mask);
sigprocmask(0, nullptr, &mask);
sigprocmask(SIG_UNBLOCK, &mask, nullptr);
execv(filename, argv);
_exit(-1);
}
int status = 0;
if (-1 == waitpid(pid, &status, 0))
{
rx.print("Cannot waitpid: %s\n", errnoToString(errno).c_str());
return -1;
}
return status;
}
void ReplxxLineReader::openEditor()
{
char filename[] = "clickhouse_replxx_XXXXXX.sql";
int fd = ::mkstemps(filename, 4);
if (-1 == fd)
{
rx.print("Cannot create temporary file to edit query: %s\n", errnoToString(errno).c_str());
return;
}
String editor = std::getenv("EDITOR");
if (editor.empty())
editor = "vim";
replxx::Replxx::State state(rx.get_state());
size_t bytes_written = 0;
const char * begin = state.text();
size_t offset = strlen(state.text());
while (bytes_written != offset)
{
ssize_t res = ::write(fd, begin + bytes_written, offset - bytes_written);
if ((-1 == res || 0 == res) && errno != EINTR)
{
rx.print("Cannot write to temporary query file %s: %s\n", filename, errnoToString(errno).c_str());
return;
}
bytes_written += res;
}
if (0 != ::close(fd))
{
rx.print("Cannot close temporary query file %s: %s\n", filename, errnoToString(errno).c_str());
return;
}
if (0 == execute(editor + " " + filename))
{
try
{
std::ifstream t(filename);
std::string str;
t.seekg(0, std::ios::end);
str.reserve(t.tellg());
t.seekg(0, std::ios::beg);
str.assign((std::istreambuf_iterator<char>(t)), std::istreambuf_iterator<char>());
rx.set_state(replxx::Replxx::State(str.c_str(), str.size()));
}
catch (...)
{
rx.print("Cannot read from temporary query file %s: %s\n", filename, errnoToString(errno).c_str());
return;
}
}
if (bracketed_paste_enabled)
enableBracketedPaste();
if (0 != ::unlink(filename))
rx.print("Cannot remove temporary query file %s: %s\n", filename, errnoToString(errno).c_str());
}
void ReplxxLineReader::enableBracketedPaste()
{
bracketed_paste_enabled = true;
rx.enable_bracketed_paste();
};

View File

@ -22,10 +22,13 @@ public:
private:
InputStatus readOneLine(const String & prompt) override;
void addToHistory(const String & line) override;
int execute(const std::string & command);
void openEditor();
replxx::Replxx rx;
replxx::Replxx::highlighter_callback_t highlighter;
// used to call flock() to synchronize multiple clients using same history file
int history_file_fd = -1;
bool bracketed_paste_enabled = false;
};

View File

@ -0,0 +1,44 @@
#include <errno.h>
#include <fcntl.h>
#include <stdint.h>
#include <stdlib.h>
#include <string.h>
#include <time.h>
#include <unistd.h>
/* This assumes that a check for the
template size has already been made */
static char * __randname(char * template)
{
int i;
struct timespec ts;
unsigned long r;
clock_gettime(CLOCK_REALTIME, &ts);
r = (ts.tv_nsec * 65537) ^ ((((intptr_t)(&ts)) / 16) + ((intptr_t)template));
for (i = 0; i < 6; i++, r >>= 5)
template[i] = 'A' + (r & 15) + (r & 16) * 2;
return template;
}
int mkstemps(char * template, int len)
{
size_t l = strlen(template);
if (l < 6 || len > l - 6 || memcmp(template + l - len - 6, "XXXXXX", 6))
{
errno = EINVAL;
return -1;
}
int fd, retries = 100;
do
{
__randname(template + l - len - 6);
if ((fd = open(template, O_RDWR | O_CREAT | O_EXCL, 0600)) >= 0)
return fd;
} while (--retries && errno == EEXIST);
memcpy(template + l - len - 6, "XXXXXX", 6);
return -1;
}

View File

@ -141,11 +141,6 @@ if(NOT EXTERNAL_PARQUET_FOUND AND NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT O
else()
set(USE_INTERNAL_PARQUET_LIBRARY 1)
if(USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE)
set(ARROW_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src")
set(PARQUET_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" ${ClickHouse_BINARY_DIR}/contrib/arrow/cpp/src)
endif()
if(MAKE_STATIC_LIBRARIES)
set(FLATBUFFERS_LIBRARY flatbuffers)
set(ARROW_LIBRARY arrow_static)
@ -155,9 +150,6 @@ if(NOT EXTERNAL_PARQUET_FOUND AND NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT O
set(FLATBUFFERS_LIBRARY flatbuffers_shared)
set(ARROW_LIBRARY arrow_shared)
set(PARQUET_LIBRARY parquet_shared)
if(USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE)
list(APPEND PARQUET_LIBRARY boost::regex)
endif()
set(THRIFT_LIBRARY thrift)
endif()

View File

@ -163,51 +163,21 @@ if(USE_INTERNAL_SNAPPY_LIBRARY)
endif()
if (USE_INTERNAL_PARQUET_LIBRARY)
if (USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE)
# We dont use arrow's cmakefiles because they uses too many depends and download some libs in compile time
# But this mode can be used for updating auto-generated parquet files:
# cmake -DUSE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE=1 -DUSE_STATIC_LIBRARIES=0
# copy {BUILD_DIR}/contrib/arrow/cpp/src/parquet/*.cpp,*.h -> /contrib/arrow-cmake/cpp/src/parquet/
# But you can update auto-generated parquet files manually:
# cd {BUILD_DIR}/contrib/arrow/cpp/src/parquet && mkdir -p build && cd build
# cmake .. -DARROW_COMPUTE=ON -DARROW_PARQUET=ON -DARROW_SIMD_LEVEL=NONE -DARROW_VERBOSE_THIRDPARTY_BUILD=ON
# -DARROW_BUILD_SHARED=1 -DARROW_BUILD_UTILITIES=OFF -DARROW_BUILD_INTEGRATION=OFF
# -DBoost_FOUND=1 -DARROW_TEST_LINKAGE="shared"
# make -j8
# copy {BUILD_DIR}/contrib/arrow/cpp/src/parquet/*.cpp,*.h -> {BUILD_DIR}/contrib/arrow-cmake/cpp/src/parquet/
# Also useful parquet reader:
# cd contrib/arrow/cpp/build && mkdir -p build && cmake .. -DPARQUET_BUILD_EXECUTABLES=1 && make -j8
# contrib/arrow/cpp/build/debug/parquet-reader some_file.parquet
# cd {BUILD_DIR}/contrib/arrow/cpp && mkdir -p build && cd build
# cmake .. -DARROW_PARQUET=1 -DARROW_WITH_SNAPPY=1 -DPARQUET_BUILD_EXECUTABLES=1
# make -j8
# {BUILD_DIR}/contrib/arrow/cpp/build/release/parquet-reader some_file.parquet
set (ARROW_COMPUTE ON CACHE INTERNAL "")
set (ARROW_PARQUET ON CACHE INTERNAL "")
set (ARROW_VERBOSE_THIRDPARTY_BUILD ON CACHE INTERNAL "")
set (ARROW_BUILD_SHARED 1 CACHE INTERNAL "")
set (ARROW_BUILD_UTILITIES OFF CACHE INTERNAL "")
set (ARROW_BUILD_INTEGRATION OFF CACHE INTERNAL "")
set (ARROW_BOOST_HEADER_ONLY ON CACHE INTERNAL "")
set (Boost_FOUND 1 CACHE INTERNAL "")
if (MAKE_STATIC_LIBRARIES)
set (PARQUET_ARROW_LINKAGE "static" CACHE INTERNAL "")
set (ARROW_TEST_LINKAGE "static" CACHE INTERNAL "")
set (ARROW_BUILD_STATIC ${MAKE_STATIC_LIBRARIES} CACHE INTERNAL "")
else ()
set (PARQUET_ARROW_LINKAGE "shared" CACHE INTERNAL "")
set (ARROW_TEST_LINKAGE "shared" CACHE INTERNAL "")
endif ()
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO")
set (_save_build_type ${CMAKE_BUILD_TYPE})
set (CMAKE_BUILD_TYPE Release)
string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
endif ()
# Because Arrow uses CMAKE_SOURCE_DIR as a project path
# Hopefully will be fixed in https://github.com/apache/arrow/pull/2676
set (CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/cmake_modules")
add_subdirectory (arrow/cpp)
if (_save_build_type)
set (CMAKE_BUILD_TYPE ${_save_build_type})
unset (_save_build_type)
string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
endif ()
else()
add_subdirectory(arrow-cmake)
# The library is large - avoid bloat.
@ -215,7 +185,6 @@ else()
target_compile_options (${THRIFT_LIBRARY} PRIVATE -g0)
target_compile_options (${PARQUET_LIBRARY} PRIVATE -g0)
endif()
endif()
if (USE_INTERNAL_AVRO_LIBRARY)
add_subdirectory(avro-cmake)

2
contrib/arrow vendored

@ -1 +1 @@
Subproject commit 3cbcb7b62c2f2d02851bff837758637eb592a64b
Subproject commit 744bdfe188f018e5e05f5deebd4e9ee0a7706cf4

View File

@ -144,15 +144,16 @@ set(ORC_SRCS
set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow)
configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_SOURCE_DIR}/cpp/src/arrow/util/config.h")
configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_BINARY_DIR}/cpp/src/arrow/util/config.h")
# arrow/cpp/src/arrow/CMakeLists.txt
set(ARROW_SRCS
${LIBRARY_DIR}/array.cc
${LIBRARY_DIR}/buffer.cc
${LIBRARY_DIR}/device.cc
${LIBRARY_DIR}/builder.cc
${LIBRARY_DIR}/chunked_array.cc
${LIBRARY_DIR}/compare.cc
${LIBRARY_DIR}/datum.cc
${LIBRARY_DIR}/device.cc
${LIBRARY_DIR}/extension_type.cc
${LIBRARY_DIR}/memory_pool.cc
${LIBRARY_DIR}/pretty_print.cc
@ -167,11 +168,12 @@ set(ARROW_SRCS
${LIBRARY_DIR}/type.cc
${LIBRARY_DIR}/visitor.cc
${LIBRARY_DIR}/tensor/coo_converter.cc
${LIBRARY_DIR}/tensor/csc_converter.cc
${LIBRARY_DIR}/tensor/csf_converter.cc
${LIBRARY_DIR}/tensor/csr_converter.cc
${LIBRARY_DIR}/array/array_base.cc
${LIBRARY_DIR}/array/array_binary.cc
${LIBRARY_DIR}/array/array_decimal.cc
${LIBRARY_DIR}/array/array_dict.cc
${LIBRARY_DIR}/array/array_nested.cc
${LIBRARY_DIR}/array/array_primitive.cc
${LIBRARY_DIR}/array/builder_adaptive.cc
${LIBRARY_DIR}/array/builder_base.cc
${LIBRARY_DIR}/array/builder_binary.cc
@ -181,17 +183,50 @@ set(ARROW_SRCS
${LIBRARY_DIR}/array/builder_primitive.cc
${LIBRARY_DIR}/array/builder_union.cc
${LIBRARY_DIR}/array/concatenate.cc
${LIBRARY_DIR}/array/dict_internal.cc
${LIBRARY_DIR}/array/data.cc
${LIBRARY_DIR}/array/diff.cc
${LIBRARY_DIR}/array/util.cc
${LIBRARY_DIR}/array/validate.cc
${LIBRARY_DIR}/csv/converter.cc
${LIBRARY_DIR}/compute/api_scalar.cc
${LIBRARY_DIR}/compute/api_vector.cc
${LIBRARY_DIR}/compute/cast.cc
${LIBRARY_DIR}/compute/exec.cc
${LIBRARY_DIR}/compute/function.cc
${LIBRARY_DIR}/compute/kernel.cc
${LIBRARY_DIR}/compute/registry.cc
${LIBRARY_DIR}/compute/kernels/aggregate_basic.cc
${LIBRARY_DIR}/compute/kernels/aggregate_mode.cc
${LIBRARY_DIR}/compute/kernels/aggregate_var_std.cc
${LIBRARY_DIR}/compute/kernels/codegen_internal.cc
${LIBRARY_DIR}/compute/kernels/scalar_arithmetic.cc
${LIBRARY_DIR}/compute/kernels/scalar_boolean.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_boolean.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_internal.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_nested.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_numeric.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_string.cc
${LIBRARY_DIR}/compute/kernels/scalar_cast_temporal.cc
${LIBRARY_DIR}/compute/kernels/scalar_compare.cc
${LIBRARY_DIR}/compute/kernels/scalar_fill_null.cc
${LIBRARY_DIR}/compute/kernels/scalar_nested.cc
${LIBRARY_DIR}/compute/kernels/scalar_set_lookup.cc
${LIBRARY_DIR}/compute/kernels/scalar_string.cc
${LIBRARY_DIR}/compute/kernels/scalar_validity.cc
${LIBRARY_DIR}/compute/kernels/vector_hash.cc
${LIBRARY_DIR}/compute/kernels/vector_nested.cc
${LIBRARY_DIR}/compute/kernels/vector_selection.cc
${LIBRARY_DIR}/compute/kernels/vector_sort.cc
${LIBRARY_DIR}/compute/kernels/util_internal.cc
${LIBRARY_DIR}/csv/chunker.cc
${LIBRARY_DIR}/csv/column_builder.cc
${LIBRARY_DIR}/csv/column_decoder.cc
${LIBRARY_DIR}/csv/converter.cc
${LIBRARY_DIR}/csv/options.cc
${LIBRARY_DIR}/csv/parser.cc
${LIBRARY_DIR}/csv/reader.cc
${LIBRARY_DIR}/csv/column_decoder.cc
${LIBRARY_DIR}/ipc/dictionary.cc
${LIBRARY_DIR}/ipc/feather.cc
@ -202,14 +237,25 @@ set(ARROW_SRCS
${LIBRARY_DIR}/ipc/writer.cc
${LIBRARY_DIR}/io/buffered.cc
${LIBRARY_DIR}/io/caching.cc
${LIBRARY_DIR}/io/compressed.cc
${LIBRARY_DIR}/io/file.cc
${LIBRARY_DIR}/io/interfaces.cc
${LIBRARY_DIR}/io/memory.cc
${LIBRARY_DIR}/io/slow.cc
${LIBRARY_DIR}/tensor/coo_converter.cc
${LIBRARY_DIR}/tensor/csf_converter.cc
${LIBRARY_DIR}/tensor/csx_converter.cc
${LIBRARY_DIR}/util/basic_decimal.cc
${LIBRARY_DIR}/util/bit_block_counter.cc
${LIBRARY_DIR}/util/bit_run_reader.cc
${LIBRARY_DIR}/util/bit_util.cc
${LIBRARY_DIR}/util/bitmap.cc
${LIBRARY_DIR}/util/bitmap_builders.cc
${LIBRARY_DIR}/util/bitmap_ops.cc
${LIBRARY_DIR}/util/bpacking.cc
${LIBRARY_DIR}/util/compression.cc
${LIBRARY_DIR}/util/compression_lz4.cc
${LIBRARY_DIR}/util/compression_snappy.cc
@ -217,8 +263,12 @@ set(ARROW_SRCS
${LIBRARY_DIR}/util/compression_zstd.cc
${LIBRARY_DIR}/util/cpu_info.cc
${LIBRARY_DIR}/util/decimal.cc
${LIBRARY_DIR}/util/delimiting.cc
${LIBRARY_DIR}/util/formatting.cc
${LIBRARY_DIR}/util/future.cc
${LIBRARY_DIR}/util/int_util.cc
${LIBRARY_DIR}/util/io_util.cc
${LIBRARY_DIR}/util/iterator.cc
${LIBRARY_DIR}/util/key_value_metadata.cc
${LIBRARY_DIR}/util/logging.cc
${LIBRARY_DIR}/util/memory.cc
@ -226,27 +276,15 @@ set(ARROW_SRCS
${LIBRARY_DIR}/util/string.cc
${LIBRARY_DIR}/util/task_group.cc
${LIBRARY_DIR}/util/thread_pool.cc
${LIBRARY_DIR}/util/time.cc
${LIBRARY_DIR}/util/trie.cc
${LIBRARY_DIR}/util/utf8.cc
${LIBRARY_DIR}/util/future.cc
${LIBRARY_DIR}/util/formatting.cc
${LIBRARY_DIR}/util/parsing.cc
${LIBRARY_DIR}/util/time.cc
${LIBRARY_DIR}/util/delimiting.cc
${LIBRARY_DIR}/util/iterator.cc
${LIBRARY_DIR}/util/value_parsing.cc
${LIBRARY_DIR}/vendored/base64.cpp
${ORC_SRCS}
)
set(ARROW_SRCS ${ARROW_SRCS}
${LIBRARY_DIR}/compute/context.cc
${LIBRARY_DIR}/compute/kernels/boolean.cc
${LIBRARY_DIR}/compute/kernels/cast.cc
${LIBRARY_DIR}/compute/kernels/hash.cc
${LIBRARY_DIR}/compute/kernels/util_internal.cc
)
if (SNAPPY_INCLUDE_DIR AND SNAPPY_LIBRARY)
set(ARROW_WITH_SNAPPY 1)
endif ()
@ -289,7 +327,8 @@ if (USE_INTERNAL_PROTOBUF_LIBRARY)
add_dependencies(${ARROW_LIBRARY} protoc)
endif ()
target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src)
target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src)
target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/cpp/src)
target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY})
target_link_libraries(${ARROW_LIBRARY} PRIVATE lz4)
if (ARROW_WITH_SNAPPY)
@ -319,19 +358,26 @@ set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/parquet)
set(GEN_LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/generated)
# arrow/cpp/src/parquet/CMakeLists.txt
set(PARQUET_SRCS
${LIBRARY_DIR}/arrow/path_internal.cc
${LIBRARY_DIR}/arrow/reader.cc
${LIBRARY_DIR}/arrow/reader_internal.cc
${LIBRARY_DIR}/arrow/schema.cc
${LIBRARY_DIR}/arrow/schema_internal.cc
${LIBRARY_DIR}/arrow/writer.cc
${LIBRARY_DIR}/arrow/path_internal.cc
${LIBRARY_DIR}/bloom_filter.cc
${LIBRARY_DIR}/column_reader.cc
${LIBRARY_DIR}/column_scanner.cc
${LIBRARY_DIR}/column_writer.cc
${LIBRARY_DIR}/deprecated_io.cc
${LIBRARY_DIR}/encoding.cc
${LIBRARY_DIR}/encryption.cc
${LIBRARY_DIR}/encryption_internal.cc
${LIBRARY_DIR}/file_reader.cc
${LIBRARY_DIR}/file_writer.cc
${LIBRARY_DIR}/internal_file_decryptor.cc
${LIBRARY_DIR}/internal_file_encryptor.cc
${LIBRARY_DIR}/level_conversion.cc
${LIBRARY_DIR}/level_comparison.cc
${LIBRARY_DIR}/metadata.cc
${LIBRARY_DIR}/murmur3.cc
${LIBRARY_DIR}/platform.cc
@ -340,10 +386,6 @@ set(PARQUET_SRCS
${LIBRARY_DIR}/schema.cc
${LIBRARY_DIR}/statistics.cc
${LIBRARY_DIR}/types.cc
${LIBRARY_DIR}/encryption.cc
${LIBRARY_DIR}/encryption_internal.cc
${LIBRARY_DIR}/internal_file_decryptor.cc
${LIBRARY_DIR}/internal_file_encryptor.cc
${GEN_LIBRARY_DIR}/parquet_constants.cpp
${GEN_LIBRARY_DIR}/parquet_types.cpp

View File

@ -1,26 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#define ARROW_VERSION_MAJOR
#define ARROW_VERSION_MINOR
#define ARROW_VERSION_PATCH
#define ARROW_VERSION ((ARROW_VERSION_MAJOR * 1000) + ARROW_VERSION_MINOR) * 1000 + ARROW_VERSION_PATCH
#define ARROW_SO_VERSION ""
#define ARROW_FULL_SO_VERSION ""
/* #undef GRPCPP_PP_INCLUDE */

View File

@ -22,8 +22,8 @@
#define PARQUET_VERSION_MINOR 5
#define PARQUET_VERSION_PATCH 1
#define PARQUET_SO_VERSION 0
#define PARQUET_FULL_SO_VERSION 0.17
#define PARQUET_SO_VERSION "200"
#define PARQUET_FULL_SO_VERSION "200.0.0"
// define the parquet created by version
#define CREATED_BY_VERSION "parquet-cpp version 1.5.1-SNAPSHOT"

View File

@ -1,5 +1,2 @@
usr/bin/clickhouse-test
usr/bin/clickhouse-test-server
usr/share/clickhouse-test/*
etc/clickhouse-client/client-test.xml
etc/clickhouse-server/server-test.xml

2
debian/rules vendored
View File

@ -93,7 +93,7 @@ override_dh_auto_build:
override_dh_auto_test:
ifeq (,$(filter nocheck,$(DEB_BUILD_OPTIONS)))
cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -E with_server
cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V
endif
override_dh_clean:

View File

@ -47,13 +47,13 @@ cp "${DOCKER_BUILD_FOLDER}/entrypoint.alpine.sh" "${CONTAINER_ROOT_FOLDER}/
## get glibc components from ubuntu 20.04 and put them to expected place
docker pull ubuntu:20.04
ubuntu20image=$(docker create --rm ubuntu:20.04)
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libc.so.6 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libdl.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libm.so.6 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libpthread.so.0 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/librt.so.1 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libnss_dns.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib/x86_64-linux-gnu/libresolv.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L ${ubuntu20image}:/lib64/ld-linux-x86-64.so.2 "${CONTAINER_ROOT_FOLDER}/lib64"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libc.so.6 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libdl.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libm.so.6 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libpthread.so.0 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/librt.so.1 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libnss_dns.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libresolv.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib64/ld-linux-x86-64.so.2 "${CONTAINER_ROOT_FOLDER}/lib64"
docker build "$DOCKER_BUILD_FOLDER" -f Dockerfile.alpine -t "yandex/clickhouse-server:${VERSION}-alpine" --pull
docker build "$DOCKER_BUILD_FOLDER" -f Dockerfile.alpine -t "yandex/clickhouse-server:${VERSION}-alpine" --pull

View File

@ -26,17 +26,17 @@ fi
CLICKHOUSE_CONFIG="${CLICKHOUSE_CONFIG:-/etc/clickhouse-server/config.xml}"
# port is needed to check if clickhouse-server is ready for connections
HTTP_PORT="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=http_port)"
HTTP_PORT="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=http_port)"
# get CH directories locations
DATA_DIR="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=path || true)"
TMP_DIR="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=tmp_path || true)"
USER_PATH="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=user_files_path || true)"
LOG_PATH="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=logger.log || true)"
LOG_DIR="$(dirname $LOG_PATH || true)"
ERROR_LOG_PATH="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=logger.errorlog || true)"
ERROR_LOG_DIR="$(dirname $ERROR_LOG_PATH || true)"
FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file $CLICKHOUSE_CONFIG --key=format_schema_path || true)"
DATA_DIR="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=path || true)"
TMP_DIR="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=tmp_path || true)"
USER_PATH="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=user_files_path || true)"
LOG_PATH="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=logger.log || true)"
LOG_DIR="$(dirname "${LOG_PATH}" || true)"
ERROR_LOG_PATH="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=logger.errorlog || true)"
ERROR_LOG_DIR="$(dirname "${ERROR_LOG_PATH}" || true)"
FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file "${CLICKHOUSE_CONFIG}" --key=format_schema_path || true)"
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
@ -92,7 +92,7 @@ fi
if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
# Listen only on localhost until the initialization is done
$gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG -- --listen_host=127.0.0.1 &
$gosu /usr/bin/clickhouse-server --config-file="${CLICKHOUSE_CONFIG}" -- --listen_host=127.0.0.1 &
pid="$!"
# check if clickhouse is ready to accept connections
@ -107,7 +107,7 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
sleep 1
done
if [ ! -z "$CLICKHOUSE_PASSWORD" ]; then
if [ -n "$CLICKHOUSE_PASSWORD" ]; then
printf -v WITH_PASSWORD '%s %q' "--password" "$CLICKHOUSE_PASSWORD"
fi
@ -130,7 +130,7 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
. "$f"
fi
;;
*.sql) echo "$0: running $f"; cat "$f" | "$clickhouseclient" ; echo ;;
*.sql) echo "$0: running $f"; "$clickhouseclient" < "$f" ; echo ;;
*.sql.gz) echo "$0: running $f"; gunzip -c "$f" | "$clickhouseclient"; echo ;;
*) echo "$0: ignoring $f" ;;
esac
@ -145,7 +145,7 @@ fi
# if no args passed to `docker run` or first argument start with `--`, then the user is passing clickhouse-server arguments
if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then
exec $gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG "$@"
exec $gosu /usr/bin/clickhouse-server --config-file="${CLICKHOUSE_CONFIG}" "$@"
fi
# Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image

View File

@ -324,7 +324,7 @@ Consider the table:
CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory();
```
When the `input_format_tsv_enum_as_number` setting is enabled:
When the `input_format_tsv_enum_as_number` setting is enabled:
```sql
SET input_format_tsv_enum_as_number = 1;
@ -1248,7 +1248,7 @@ Consider the table:
CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory();
```
When the `input_format_csv_enum_as_number` setting is enabled:
When the `input_format_csv_enum_as_number` setting is enabled:
```sql
SET input_format_csv_enum_as_number = 1;
@ -1841,7 +1841,7 @@ Default value: 0.
Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table.
By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).
By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).
Possible values:
@ -2109,8 +2109,8 @@ Enables [ORDER BY](../../sql-reference/statements/select/order-by.md#optimize_re
Possible values:
- 0 — `ORDER BY` optimization is disabled.
- 1 — `ORDER BY` optimization is enabled.
- 0 — `ORDER BY` optimization is disabled.
- 1 — `ORDER BY` optimization is enabled.
Default value: `1`.
@ -2124,8 +2124,8 @@ Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql
Possible values:
- 0 - Mutations execute asynchronously.
- 1 - The query waits for all mutations to complete on the current server.
- 0 - Mutations execute asynchronously.
- 1 - The query waits for all mutations to complete on the current server.
- 2 - The query waits for all mutations to complete on all replicas (if they exist).
Default value: `0`.
@ -2137,11 +2137,11 @@ Default value: `0`.
## ttl_only_drop_parts {#ttl_only_drop_parts}
Enables or disables complete dropping of data parts where all rows are expired in [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables.
Enables or disables complete dropping of data parts where all rows are expired in [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables.
When `ttl_only_drop_parts` is disabled (by default), the ClickHouse server only deletes expired rows according to their TTL.
When `ttl_only_drop_parts` is disabled (by default), the ClickHouse server only deletes expired rows according to their TTL.
When `ttl_only_drop_parts` is enabled, the ClickHouse server drops a whole part when all rows in it are expired.
When `ttl_only_drop_parts` is enabled, the ClickHouse server drops a whole part when all rows in it are expired.
Dropping whole parts instead of partial cleaning TTL-d rows allows having shorter `merge_with_ttl_timeout` times and lower impact on system performance.
@ -2152,14 +2152,14 @@ Possible values:
Default value: `0`.
**See Also**
**See Also**
- [CREATE TABLE query clauses and settings](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) (`merge_with_ttl_timeout` setting)
- [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl)
## lock_acquire_timeout {#lock_acquire_timeout}
Defines how many seconds a locking request waits before failing.
Defines how many seconds a locking request waits before failing.
Locking timeout is used to protect from deadlocks while executing read/write operations with tables. When the timeout expires and the locking request fails, the ClickHouse server throws an exception "Locking attempt timed out! Possible deadlock avoided. Client should retry." with error code `DEADLOCK_AVOIDED`.
@ -2179,11 +2179,11 @@ When the setting is enabled and the argument of `CAST` function is `Nullable`, t
Possible values:
- 0 — The `CAST` result has exactly the destination type specified.
- 1 — If the argument type is `Nullable`, the `CAST` result is transformed to `Nullable(DestinationDataType)`.
- 1 — If the argument type is `Nullable`, the `CAST` result is transformed to `Nullable(DestinationDataType)`.
Default value: `0`.
**Examples**
**Examples**
The following query results in the destination data type exactly:
@ -2215,17 +2215,17 @@ Result:
└───┴───────────────────────────────────────────────────┘
```
**See Also**
**See Also**
- [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function
## output_format_pretty_max_value_width {#output_format_pretty_max_value_width}
Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the limit, the value is cut.
Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the limit, the value is cut.
Possible values:
- Positive integer.
- Positive integer.
- 0 — The value is cut completely.
Default value: `10000` symbols.
@ -2352,7 +2352,7 @@ Default value: `0`.
## persistent {#persistent}
Disables persistency for the [Set](../../engines/table-engines/special/set.md#set) and [Join](../../engines/table-engines/special/join.md#join) table engines.
Disables persistency for the [Set](../../engines/table-engines/special/set.md#set) and [Join](../../engines/table-engines/special/join.md#join) table engines.
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
@ -2426,7 +2426,7 @@ Result:
[
{"number":"0"},
{"number":"1"},
{"number":"2"}
{"number":"2"}
]
```
@ -2447,7 +2447,6 @@ Result:
{"number":"2"}
```
=======
## allow_nullable_key {#allow-nullable-key}
Allows using of the [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) tables.

View File

@ -9,7 +9,6 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
Поддерживаемые форматы и возможность использовать их в запросах `INSERT` и `SELECT` перечислены в таблице ниже.
=======
| Формат | INSERT | SELECT |
|-----------------------------------------------------------------------------------------|--------|--------|
| [TabSeparated](#tabseparated) | ✔ | ✔ |
@ -1252,7 +1251,7 @@ SELECT * FROM line_as_string;
## RawBLOB {#rawblob}
В этом формате все входные данные считываются в одно значение. Парсить можно только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему.
В этом формате все входные данные считываются в одно значение. Парсить можно только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему.
Результат выводится в бинарном виде без разделителей и экранирования. При выводе более одного значения формат неоднозначен и будет невозможно прочитать данные снова.
Ниже приведено сравнение форматов `RawBLOB` и [TabSeparatedRaw](#tabseparatedraw).
@ -1272,7 +1271,7 @@ SELECT * FROM line_as_string;
- строки представлены как длина в формате varint (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), а затем байты строки.
При передаче на вход `RawBLOB` пустых данных, ClickHouse бросает исключение:
``` text
Code: 108. DB::Exception: No data to insert
```

View File

@ -363,7 +363,7 @@ endif ()
if (USE_PARQUET)
dbms_target_link_libraries(PRIVATE ${PARQUET_LIBRARY})
if (NOT USE_INTERNAL_PARQUET_LIBRARY OR USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE)
if (NOT USE_INTERNAL_PARQUET_LIBRARY)
dbms_target_include_directories (SYSTEM BEFORE PRIVATE ${PARQUET_INCLUDE_DIR} ${ARROW_INCLUDE_DIR})
if (USE_STATIC_LIBRARIES)
dbms_target_link_libraries(PRIVATE ${ARROW_LIBRARY})

View File

@ -414,7 +414,9 @@ class IColumn;
M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \
M(Bool, enable_debug_queries, false, "Enabled debug queries, but now is obsolete", 0) \
M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \
M(UnionMode, union_default_mode, UnionMode::DISTINCT, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0)
M(UnionMode, union_default_mode, UnionMode::DISTINCT, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below.
@ -426,10 +428,10 @@ class IColumn;
M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(Bool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \
M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices \\N", 0) \
M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \
M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices \\N", 0) \
@ -438,7 +440,6 @@ class IColumn;
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
\
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \

View File

@ -300,7 +300,7 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe
template <typename Reader>
static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && read_nested)
static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && read_nested, bool allow_unenclosed)
{
ColumnArray & column_array = assert_cast<ColumnArray &>(column);
ColumnArray::Offsets & offsets = column_array.getOffsets();
@ -308,7 +308,12 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
IColumn & nested_column = column_array.getData();
size_t size = 0;
assertChar('[', istr);
bool has_braces = false;
if (checkChar('[', istr))
has_braces = true;
else if (!allow_unenclosed)
throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Array does not start with '[' character");
try
{
@ -320,7 +325,9 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
if (*istr.position() == ',')
++istr.position();
else
throw Exception("Cannot read array from text", ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT);
throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT,
"Cannot read array from text, expected comma or end of array, found '{}'",
*istr.position());
}
first = false;
@ -335,7 +342,11 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
skipWhitespaceIfAny(istr);
}
assertChar(']', istr);
if (has_braces)
assertChar(']', istr);
else /// If array is not enclosed in braces, we read until EOF.
assertEOF(istr);
}
catch (...)
{
@ -364,7 +375,7 @@ void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr, const F
[&](IColumn & nested_column)
{
nested->deserializeAsTextQuoted(nested_column, istr, settings);
});
}, false);
}
void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -390,7 +401,11 @@ void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, Wr
void DataTypeArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeTextImpl(column, istr, [&](IColumn & nested_column) { nested->deserializeAsTextJSON(nested_column, istr, settings); });
deserializeTextImpl(column, istr,
[&](IColumn & nested_column)
{
nested->deserializeAsTextJSON(nested_column, istr, settings);
}, false);
}
@ -429,7 +444,23 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons
String s;
readCSV(s, istr, settings.csv);
ReadBufferFromString rb(s);
deserializeText(column, rb, settings);
if (settings.csv.input_format_arrays_as_nested_csv)
{
deserializeTextImpl(column, rb,
[&](IColumn & nested_column)
{
nested->deserializeAsTextCSV(nested_column, rb, settings);
}, true);
}
else
{
deserializeTextImpl(column, rb,
[&](IColumn & nested_column)
{
nested->deserializeAsTextQuoted(nested_column, rb, settings);
}, true);
}
}

View File

@ -64,6 +64,7 @@ FormatSettings getFormatSettings(const Context & context,
format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields;
format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number;
format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null;
format_settings.csv.input_format_arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv;
format_settings.custom.escaping_rule = settings.format_custom_escaping_rule;
format_settings.custom.field_delimiter = settings.format_custom_field_delimiter;
format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter;

View File

@ -71,6 +71,7 @@ struct FormatSettings
bool empty_as_default = false;
bool crlf_end_of_line = false;
bool input_format_enum_as_number = false;
bool input_format_arrays_as_nested_csv = false;
} csv;
struct Custom

View File

@ -1216,7 +1216,10 @@ public:
{
return res;
}
else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))
else if ((isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))
// Comparing Date and DateTime64 requires implicit conversion,
// otherwise Date is treated as number.
&& !(date_and_datetime && (isDate(left_type) || isDate(right_type))))
{
// compare
if (!allowDecimalComparison(left_type, right_type) && !date_and_datetime)

View File

@ -62,9 +62,9 @@ void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr<arrow::Schema>
// TODO: should we use arrow::ipc::IpcOptions::alignment?
if (stream)
writer_status = arrow::ipc::NewStreamWriter(arrow_ostream.get(), schema);
writer_status = arrow::ipc::MakeStreamWriter(arrow_ostream.get(), schema);
else
writer_status = arrow::ipc::NewFileWriter(arrow_ostream.get(), schema);
writer_status = arrow::ipc::MakeFileWriter(arrow_ostream.get(), schema);
if (!writer_status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION,

View File

@ -386,16 +386,17 @@ static void appendBlock(const Block & from, Block & to)
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
MutableColumnPtr last_col;
try
{
for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no)
{
const IColumn & col_from = *from.getByPosition(column_no).column.get();
MutableColumnPtr col_to = IColumn::mutate(std::move(to.getByPosition(column_no).column));
last_col = IColumn::mutate(std::move(to.getByPosition(column_no).column));
col_to->insertRangeFrom(col_from, 0, rows);
last_col->insertRangeFrom(col_from, 0, rows);
to.getByPosition(column_no).column = std::move(col_to);
to.getByPosition(column_no).column = std::move(last_col);
}
}
catch (...)
@ -406,6 +407,16 @@ static void appendBlock(const Block & from, Block & to)
for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no)
{
ColumnPtr & col_to = to.getByPosition(column_no).column;
/// If there is no column, then the exception was thrown in the middle of append, in the insertRangeFrom()
if (!col_to)
{
col_to = std::move(last_col);
/// Suppress clang-tidy [bugprone-use-after-move]
last_col = {};
}
/// But if there is still nothing, abort
if (!col_to)
throw Exception("No column to rollback", ErrorCodes::LOGICAL_ERROR);
if (col_to->size() != old_rows)
col_to = col_to->cut(0, old_rows);
}

View File

@ -7,7 +7,7 @@ else ()
include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake)
endif ()
install (PROGRAMS clickhouse-test clickhouse-test-server DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
install (
DIRECTORY queries performance config
DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test
@ -17,30 +17,6 @@ install (
PATTERN ".gitignore" EXCLUDE
)
install (FILES server-test.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse)
install (FILES client-test.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-client COMPONENT clickhouse)
if (ENABLE_TESTS)
configure_file(${CMAKE_CURRENT_SOURCE_DIR}/CTestCustom.cmake ${ClickHouse_BINARY_DIR})
# maybe add --no-long ?
# if you want disable some tests: env TEST_OPT0='--skip compile'
if(TEST_CMAKE_PARALLEL)
# NUMBER_OF_LOGICAL_CORES
if (TEST_CMAKE_PARALLEL GREATER 1)
set(TOTAL_TESTS ${TEST_CMAKE_PARALLEL})
else()
set(TOTAL_TESTS ${NUMBER_OF_LOGICAL_CORES})
endif()
foreach(proc RANGE 1 ${TOTAL_TESTS})
add_test(NAME with_server${proc} COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} TEST_OPT0=--parallel=${proc}/${TOTAL_TESTS} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endforeach()
else()
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endif()
endif ()
if (ENABLE_TEST_INTEGRATION)
add_subdirectory (integration)
endif ()

View File

@ -1,5 +0,0 @@
set(CTEST_CUSTOM_TESTS_IGNORE
example
example64
capnp-heavy-tests-run
)

View File

@ -1,3 +0,0 @@
<yandex>
<insert_format_max_block_size>100000</insert_format_max_block_size>
</yandex>

View File

@ -1,166 +0,0 @@
#!/usr/bin/env bash
set -x
set -o errexit
set -o pipefail
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
ROOT_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && cd ../.. && pwd)
DATA_DIR=${DATA_DIR:=`mktemp -d /tmp/clickhouse.test..XXXXX`}
DATA_DIR_PATTERN=${DATA_DIR_PATTERN:=/tmp/clickhouse} # path from config file, will be replaced to temporary
LOG_DIR=${LOG_DIR:=$DATA_DIR/log}
export CLICKHOUSE_BINARY_NAME=${CLICKHOUSE_BINARY_NAME:="clickhouse"}
( [ -x "$ROOT_DIR/programs/${CLICKHOUSE_BINARY_NAME}-server" ] || [ -x "$ROOT_DIR/programs/${CLICKHOUSE_BINARY_NAME}" ] ) && BUILD_DIR=${BUILD_DIR:=$ROOT_DIR} # Build without separate build dir
[ -d "$ROOT_DIR/build${BUILD_TYPE}" ] && BUILD_DIR=${BUILD_DIR:=$ROOT_DIR/build${BUILD_TYPE}}
BUILD_DIR=${BUILD_DIR:=$ROOT_DIR}
[ -x ${CLICKHOUSE_BINARY_NAME}-server" ] && [ -x ${CLICKHOUSE_BINARY_NAME}-client" ] && BIN_DIR= # Allow run in /usr/bin
( [ -x "$BUILD_DIR/programs/${CLICKHOUSE_BINARY_NAME}" ] || [ -x "$BUILD_DIR/programs/${CLICKHOUSE_BINARY_NAME}-server" ] ) && BIN_DIR=${BIN_DIR:=$BUILD_DIR/programs/}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-server" ] && CLICKHOUSE_SERVER=${CLICKHOUSE_SERVER:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-server}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}" ] && CLICKHOUSE_SERVER=${CLICKHOUSE_SERVER:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME} server}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-client" ] && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-client}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}" ] && CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME} client}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-extract-from-config" ] && CLICKHOUSE_EXTRACT=${CLICKHOUSE_EXTRACT:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME}-extract-from-config}
[ -x "$BIN_DIR/${CLICKHOUSE_BINARY_NAME}" ] && CLICKHOUSE_EXTRACT=${CLICKHOUSE_EXTRACT:=$BIN_DIR/${CLICKHOUSE_BINARY_NAME} extract-from-config}
[ -f "$CUR_DIR/server-test.xml" ] && CONFIG_DIR=${CONFIG_DIR=$CUR_DIR}/
CONFIG_CLIENT_DIR=${CONFIG_CLIENT_DIR=$CONFIG_DIR}
CONFIG_SERVER_DIR=${CONFIG_SERVER_DIR=$CONFIG_DIR}
[ ! -f "${CONFIG_CLIENT_DIR}client-test.xml" ] && CONFIG_CLIENT_DIR=${CONFIG_CLIENT_DIR:=/etc/clickhouse-client/}
[ ! -f "${CONFIG_SERVER_DIR}server-test.xml" ] && CONFIG_SERVER_DIR=${CONFIG_SERVER_DIR:=/etc/clickhouse-server/}
export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:=${CONFIG_CLIENT_DIR}client-test.xml}
export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:=${CONFIG_SERVER_DIR}server-test.xml}
CLICKHOUSE_CONFIG_USERS=${CONFIG_SERVER_DIR}users.xml
[ ! -f "$CLICKHOUSE_CONFIG_USERS" ] && CLICKHOUSE_CONFIG_USERS=$CUR_DIR/../programs/server/users.xml
CLICKHOUSE_CONFIG_USERS_D=${CONFIG_SERVER_DIR}users.d
[ ! -d "$CLICKHOUSE_CONFIG_USERS_D" ] && CLICKHOUSE_CONFIG_USERS_D=$CUR_DIR/../programs/server/users.d
[ -x "$CUR_DIR/clickhouse-test" ] && TEST_DIR=${TEST_DIR=$CUR_DIR/}
[ -d "$CUR_DIR/queries" ] && QUERIES_DIR=${QUERIES_DIR=$CUR_DIR/queries}
[ ! -d "$QUERIES_DIR" ] && [ -d "/usr/local/share/clickhouse-test/queries" ] && QUERIES_DIR=${QUERIES_DIR=/usr/local/share/clickhouse-test/queries}
[ ! -d "$QUERIES_DIR" ] && [ -d "/usr/share/clickhouse-test/queries" ] && QUERIES_DIR=${QUERIES_DIR=/usr/share/clickhouse-test/queries}
TEST_PORT_RANDOM=${TEST_PORT_RANDOM=1}
if [ "${TEST_PORT_RANDOM}" ]; then
CLICKHOUSE_PORT_BASE=${CLICKHOUSE_PORT_BASE:=$(( ( RANDOM % 50000 ) + 10000 ))}
CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=$(($CLICKHOUSE_PORT_BASE + 1))}
CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=$(($CLICKHOUSE_PORT_BASE + 2))}
CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:=$(($CLICKHOUSE_PORT_BASE + 3))}
CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=$(($CLICKHOUSE_PORT_BASE + 4))}
CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=$(($CLICKHOUSE_PORT_BASE + 5))}
CLICKHOUSE_PORT_ODBC_BRIDGE=${CLICKHOUSE_ODBC_BRIDGE:=$(($CLICKHOUSE_PORT_BASE + 6))}
fi
rm -rf $DATA_DIR || true
mkdir -p $LOG_DIR $DATA_DIR/etc || true
if [ "$DATA_DIR_PATTERN" != "$DATA_DIR" ]; then
cat $CLICKHOUSE_CONFIG | sed -e s!$DATA_DIR_PATTERN!$DATA_DIR! > $DATA_DIR/etc/server-config.xml
export CLICKHOUSE_CONFIG=$DATA_DIR/etc/server-config.xml
cp $CLICKHOUSE_CONFIG_USERS $DATA_DIR/etc
cp -R -L $CLICKHOUSE_CONFIG_USERS_D $DATA_DIR/etc
cat ${CONFIG_SERVER_DIR}/ints_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/ints_dictionary.xml
cat ${CONFIG_SERVER_DIR}/strings_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/strings_dictionary.xml
cat ${CONFIG_SERVER_DIR}/decimals_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/decimals_dictionary.xml
fi
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${CLICKHOUSE_EXTRACT} --config=$CLICKHOUSE_CONFIG"}
CLICKHOUSE_LOG=${CLICKHOUSE_LOG:=${LOG_DIR}clickhouse-server.log}
export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port || echo 9000`}
export CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=http_port || echo 8123`}
export CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:=`$CLICKHOUSE_EXTRACT_CONFIG --key=interserver_http_port || echo 9009`}
export CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=`$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port_secure`}
export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=`$CLICKHOUSE_EXTRACT_CONFIG --key=https_port`}
export CLICKHOUSE_ODBC_BRIDGE=${CLICKHOUSE_ODBC_BRIDGE:=`$CLICKHOUSE_EXTRACT_CONFIG --key=odbc_bridge.port || echo 9018`}
DHPARAM=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.dhParamsFile`
PRIVATEKEY=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.privateKeyFile`
CERT=`$CLICKHOUSE_EXTRACT_CONFIG --key=openSSL.server.certificateFile`
# Do not generate in case broken extract-config
[ -n "$DHPARAM" ] && openssl dhparam -out $DHPARAM 256
[ -n "$PRIVATEKEY" ] && [ -n "$CERT" ] && openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout $PRIVATEKEY -out $CERT
if [ "$TEST_GDB" ] || [ "$GDB" ]; then
echo -e "run \nset pagination off \nset logging file $LOG_DIR/server.gdb.log \nset logging on \nbacktrace \nthread apply all backtrace \nbacktrace \ndetach \nquit " > $DATA_DIR/gdb.cmd
GDB=${GDB:="gdb -x $DATA_DIR/gdb.cmd --args "}
fi
# Start a local clickhouse server which will be used to run tests
# TODO: fix change shard ports:
# --remote_servers.test_shard_localhost_secure.shard.replica.port=$CLICKHOUSE_PORT_TCP_SECURE \
# --remote_servers.test_shard_localhost.shard.replica.port=$CLICKHOUSE_PORT_TCP \
VERSION=`$CLICKHOUSE_CLIENT --version-clean`
# If run from compile dir - use in-place compile binary and headers
[ -n "$BIN_DIR" ] && INTERNAL_COMPILER_PARAMS="--compiler_executable_root=${INTERNAL_COMPILER_BIN_ROOT:=$BUILD_DIR/programs/} --compiler_headers=$BUILD_DIR/programs/clang/headers/$VERSION/ --compiler_headers_root=$BUILD_DIR/programs/clang/headers/$VERSION/"
$GDB $CLICKHOUSE_SERVER --config-file=$CLICKHOUSE_CONFIG --log=$CLICKHOUSE_LOG $TEST_SERVER_PARAMS -- \
--http_port=$CLICKHOUSE_PORT_HTTP \
--tcp_port=$CLICKHOUSE_PORT_TCP \
--https_port=$CLICKHOUSE_PORT_HTTPS \
--tcp_port_secure=$CLICKHOUSE_PORT_TCP_SECURE \
--interserver_http_port=$CLICKHOUSE_PORT_INTERSERVER \
--odbc_bridge.port=$CLICKHOUSE_ODBC_BRIDGE \
$INTERNAL_COMPILER_PARAMS \
$TEST_SERVER_CONFIG_PARAMS \
2>&1 > $LOG_DIR/server.stdout.log &
CH_PID=$!
sleep ${TEST_SERVER_STARTUP_WAIT:=5}
if [ "$GDB" ]; then
# Long symbols read
sleep ${TEST_GDB_SLEEP:=60}
fi
tail -n50 $LOG_DIR/*.log || true
# Define needed stuff to kill test clickhouse server after tests completion
function finish {
kill $CH_PID || true
wait
tail -n 50 $LOG_DIR/*.log || true
if [ "$GDB" ]; then
cat $LOG_DIR/server.gdb.log || true
fi
rm -rf $DATA_DIR
}
trap finish EXIT SIGINT SIGQUIT SIGTERM
# Do tests
if [ -n "$*" ]; then
$*
else
TEST_RUN=${TEST_RUN=1}
TEST_DICT=${TEST_DICT=1}
CLICKHOUSE_CLIENT_QUERY="${CLICKHOUSE_CLIENT} --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
$CLICKHOUSE_CLIENT_QUERY 'SELECT * from system.build_options; SELECT * FROM system.clusters;'
CLICKHOUSE_TEST="env ${TEST_DIR}clickhouse-test --force-color --binary ${BIN_DIR}${CLICKHOUSE_BINARY_NAME} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
if [ "${TEST_RUN_STRESS}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)
TEST_NPROC=${TEST_NPROC:=$(( `nproc || sysctl -n hw.ncpu || echo 2` * 2))}
for i in `seq 1 ${TEST_NPROC}`; do
$CLICKHOUSE_TEST --order=random --testname --tmp=$DATA_DIR/tmp/tmp${i} &
done
fi
if [ "${TEST_RUN_PARALLEL}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)
TEST_NPROC=${TEST_NPROC:=$(( `nproc || sysctl -n hw.ncpu || echo 2` * 2))}
for i in `seq 1 ${TEST_NPROC}`; do
$CLICKHOUSE_TEST --testname --tmp=$DATA_DIR/tmp/tmp${i} --database=test${i} --parallel=${i}/${TEST_NPROC} &
done
for job in `jobs -p`; do
#echo wait $job
wait $job || let "FAIL+=1"
done
#echo $FAIL
if [ "$FAIL" != "0" ]; then
return $FAIL
fi
else
( [ "$TEST_RUN" ] && $CLICKHOUSE_TEST ) || ${TEST_TRUE:=false}
fi
$CLICKHOUSE_CLIENT_QUERY "SELECT event, value FROM system.events; SELECT metric, value FROM system.metrics; SELECT metric, value FROM system.asynchronous_metrics;"
$CLICKHOUSE_CLIENT_QUERY "SELECT 'Still alive'"
fi

View File

@ -1,17 +0,0 @@
<!-- Config for connecting to test server -->
<yandex>
<tcp_port>59000</tcp_port>
<tcp_port_secure>59440</tcp_port_secure>
<openSSL>
<client>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -0,0 +1,36 @@
<test>
<settings>
<allow_experimental_map_type>1</allow_experimental_map_type>
</settings>
<substitutions>
<substitution>
<name>key_suffix</name>
<values>
<value>''</value>
<value>'-miss'</value>
</values>
</substitution>
</substitutions>
<create_query>
CREATE TABLE column_map_test
ENGINE = MergeTree ORDER BY number
AS
SELECT number, map
FROM
(
SELECT
number,
arrayMap(x -> toString(x), range(100)) AS keys,
arrayMap(x -> toString(x * x), range(100)) AS values,
cast((keys, values), 'Map(String, String)') AS map
FROM numbers(10000)
)
</create_query>
<query>SELECT count() FROM column_map_test WHERE NOT ignore(arrayMap(x -> map[CONCAT(toString(x), {key_suffix})], range(0, 100, 10)))</query>
<drop_query>DROP TABLE IF EXISTS column_map_test</drop_query>
</test>

View File

@ -1,54 +0,0 @@
#!/usr/bin/env perl
package parquet_create_table_columns;
use strict;
no warnings 'experimental';
use feature 'signatures';
use JSON::XS;
#use Data::Dumper;
sub file_read($file) {
open my $f, '<', $file or return;
local $/ = undef;
my $ret = <$f>;
close $f;
return $ret;
}
our $type_parquet_logical_to_clickhouse = {
DECIMAL => 'Decimal128(1)',
TIMESTAMP_MICROS => 'DateTime',
TIMESTAMP_MILLIS => 'DateTime',
};
our $type_parquet_physical_to_clickhouse = {
BOOLEAN => 'UInt8',
INT32 => 'Int32',
INT64 => 'Int64',
FLOAT => 'Float32',
DOUBLE => 'Float64',
BYTE_ARRAY => 'String',
FIXED_LEN_BYTE_ARRAY => 'String', # Maybe FixedString?
INT96 => 'Int64', # TODO!
};
sub columns ($json) {
my @list;
my %uniq;
for my $column (@{$json->{Columns}}) {
#warn Data::Dumper::Dumper $column;
my $name = $column->{'Name'};
my $type = $type_parquet_logical_to_clickhouse->{$column->{'LogicalType'}} || $type_parquet_physical_to_clickhouse->{$column->{'PhysicalType'}};
unless ($type) {
warn "Unknown type [$column->{'PhysicalType'}:$column->{'LogicalType'}] of column [$name]";
}
$type = "Nullable($type)";
$name .= $column->{'Id'} if $uniq{$name}++; # Names can be non-unique
push @list, {name => $name, type => $type};
}
print join ', ', map {"`$_->{name}` $_->{type}"} @list;
}
sub columns_file ($file) {
return columns(JSON::XS::decode_json(file_read($file)));
}
columns_file(shift) unless caller;

View File

@ -13,134 +13,220 @@
=== Try load data from alltypes_plain.snappy.parquet
6 1 0 0 0 0 0 0 04/01/09 0 1238544000
7 0 1 1 1 10 1.1 10.1 04/01/09 1 1238544060
=== Try load data from binary.parquet
\0







\b
\t
\n
=== Try load data from byte_array_decimal.parquet
1.0
2.0
3.0
4.0
5.0
6.0
7.0
8.0
9.0
10.0
11.0
12.0
13.0
14.0
15.0
16.0
17.0
18.0
19.0
20.0
21.0
22.0
23.0
24.0
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from datapage_v2.snappy.parquet
Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding.: data for INSERT was parsed from stdin
=== Try load data from dict-page-offset-zero.parquet
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
1552
=== Try load data from fixed_length_decimal.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from fixed_length_decimal_1.parquet
1.0
2.0
3.0
4.0
5.0
6.0
7.0
8.0
9.0
10.0
11.0
12.0
13.0
14.0
15.0
16.0
17.0
18.0
19.0
20.0
21.0
22.0
23.0
24.0
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from fixed_length_decimal_legacy.parquet
1.0
2.0
3.0
4.0
5.0
6.0
7.0
8.0
9.0
10.0
11.0
12.0
13.0
14.0
15.0
16.0
17.0
18.0
19.0
20.0
21.0
22.0
23.0
24.0
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from hadoop_lz4_compressed.parquet
1593604800 abc 42
1593604800 def 7.7
1593604801 abc 42.125
1593604801 def 7.7
=== Try load data from int32_decimal.parquet
1.0
2.0
3.0
4.0
5.0
6.0
7.0
8.0
9.0
10.0
11.0
12.0
13.0
14.0
15.0
16.0
17.0
18.0
19.0
20.0
21.0
22.0
23.0
24.0
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from int64_decimal.parquet
1.0
2.0
3.0
4.0
5.0
6.0
7.0
8.0
9.0
10.0
11.0
12.0
13.0
14.0
15.0
16.0
17.0
18.0
19.0
20.0
21.0
22.0
23.0
24.0
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
=== Try load data from list_columns.parquet
Code: 70. DB::Ex---tion: The type "list" of an input column "int64_list" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
=== Try load data from nation.dict-malformed.parquet
0 ALGERIA 0 haggle. carefully final deposits detect slyly agai
1 ARGENTINA 1 al foxes promise slyly according to the regular accounts. bold requests alon
@ -168,23 +254,25 @@ Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet impl
23 UNITED KINGDOM 3 eans boost carefully special requests. accounts are. carefull
24 UNITED STATES 1 y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be
=== Try load data from nested_lists.snappy.parquet
Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin
Code: 70. DB::Ex---tion: The type "list" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
=== Try load data from nested_maps.snappy.parquet
Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Reading lists of structs from Parquet files not yet supported: key_value: list<key_value: struct<key: string not null, value: struct<key_value: list<key_value: struct<key: int32 not null, value: bool not null> not null> not null>> not null> not null: data for INSERT was parsed from stdin
Code: 70. DB::Ex---tion: The type "map" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
=== Try load data from non_hadoop_lz4_compressed.parquet
1593604800 abc 42
1593604800 def 7.7
1593604801 abc 42.125
1593604801 def 7.7
=== Try load data from nonnullable.impala.parquet
Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin
../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id())
=== Try load data from nullable.impala.parquet
Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin
../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id())
=== Try load data from nulls.snappy.parquet
Code: 8. DB::Ex---tion: Column "b_c_int" is not presented in input data: data for INSERT was parsed from stdin
=== Try load data from repeated_no_annotation.parquet
Code: 8. DB::Ex---tion: Column "number" is not presented in input data: data for INSERT was parsed from stdin
Code: 70. DB::Ex---tion: The type "struct" of an input column "b_struct" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
=== Try load data from single_nan.parquet
\N
=== Try load data from userdata1.parquet
1454486129 1 Amanda Jordan ajordan0@com.com Female 1.197.201.2 6759521864920116 Indonesia 3/8/1971 49756.53 Internal Auditor 1E+02
1454519043 2 Albert Freeman afreeman1@is.gd Male 218.111.175.34 Canada 1/16/1968 150280.17 Accountant IV

View File

@ -5,8 +5,6 @@
# TODO: Add more files.
#
# To regenerate data install perl JSON::XS module: sudo apt install libjson-xs-perl
# Also 5 sample files from
# wget https://github.com/Teradata/kylo/raw/master/samples/sample-data/parquet/userdata1.parquet
# ...
@ -19,38 +17,46 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
CB_DIR=$(dirname "$CLICKHOUSE_CLIENT_BINARY")
[ "$CB_DIR" == "." ] && ROOT_DIR=$CUR_DIR/../../../..
[ "$CB_DIR" != "." ] && BUILD_DIR=$CB_DIR/../..
[ -z "$ROOT_DIR" ] && ROOT_DIR=$CB_DIR/../../..
[ "$CB_DIR" == "." ] && ROOT_DIR=$CUR_DIR/../../..
[ -z "$ROOT_DIR" ] && ROOT_DIR=$CB_DIR/../..
DATA_DIR=$CUR_DIR/data_parquet
[ -n "$ROOT_DIR" ] && [ -z "$PARQUET_READER" ] && PARQUET_READER="$ROOT_DIR"/contrib/arrow/cpp/build/release/parquet-reader
# To update:
# cp $ROOT_DIR/contrib/arrow/cpp/submodules/parquet-testing/data/*.parquet $ROOT_DIR/contrib/arrow/python/pyarrow/tests/data/parquet/*.parquet $CUR_DIR/data_parquet/
# BUG! nulls.snappy.parquet - parquet-reader shows wrong structure. Actual structure is {"type":"struct","fields":[{"name":"b_struct","type":{"type":"struct","fields":[{"name":"b_c_int","type":"integer","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]}
# why? repeated_no_annotation.parquet
# ClickHouse Parquet reader doesn't support such complex types, so I didn't burrow into the issue.
# There is failure due parsing nested arrays or nested maps with NULLs:
# ../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id())
for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | sort); do
# Strange behaviour for repeated_no_annotation.parquet around __buitin_expect, so this file was disabled:
# debug:
# ../contrib/arrow/cpp/src/arrow/array/array_nested.cc:193: Check failed: self->list_type_->value_type()->Equals(data->child_data[0]->type)
# release:
# Code: 349. DB::Ex---tion: Can not insert NULL data into non-nullable column "phoneNumbers": data for INSERT was parsed from stdin
for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_ALL=C sort); do
echo === Try load data from "$NAME"
JSON=$DATA_DIR/$NAME.json
COLUMNS_FILE=$DATA_DIR/$NAME.columns
# If you want change or add .parquet file - rm data_parquet/*.json data_parquet/*.columns
[ -n "$BUILD_DIR" ] && [ ! -s "$COLUMNS_FILE" ] && [ ! -s "$JSON" ] && "$BUILD_DIR"/contrib/arrow-cmake/parquet-reader --json "$DATA_DIR"/"$NAME" > "$JSON"
[ -n "$BUILD_DIR" ] && [ ! -s "$COLUMNS_FILE" ] && "$CUR_DIR"/00900_parquet_create_table_columns.pl "$JSON" > "$COLUMNS_FILE"
[ -n "$PARQUET_READER" ] && [ ! -s "$COLUMNS_FILE" ] && [ ! -s "$JSON" ] && "$PARQUET_READER" --json "$DATA_DIR"/"$NAME" > "$JSON"
[ ! -s "$COLUMNS_FILE" ] && "$CUR_DIR"/helpers/00900_parquet_create_table_columns.py "$JSON" > "$COLUMNS_FILE"
# Debug only:
# [ -n "$BUILD_DIR" ] && $BUILD_DIR/contrib/arrow-cmake/parquet-reader $DATA_DIR/$NAME > $DATA_DIR/$NAME.dump
# [ -n "$PARQUET_READER" ] && $PARQUET_READER $DATA_DIR/$NAME > $DATA_DIR/$NAME.dump
#COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.pl $JSON` 2>&1 || continue
# COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.py $JSON` 2>&1 || continue
COLUMNS=$(cat "$COLUMNS_FILE") || continue
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory"
# Some files is broken, exception is ok.
# Some files contain unsupported data structures, exception is ok.
cat "$DATA_DIR"/"$NAME" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" 2>&1 | sed 's/Exception/Ex---tion/'
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load LIMIT 100"

View File

@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)
yes 'SELECT 1' 2>/dev/null | {
head -n1000
} | {
xargs -i ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((1<<30))" -d '{}'
xargs -I{} ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((1<<30))" -d '{}'
} | grep -x -c 1
wait

View File

@ -5,7 +5,7 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
. "$CURDIR"/../shell_config.sh
function test()
function test_func()
{
ENGINE=$1
MAX_MEM=4096
@ -32,9 +32,9 @@ function test()
$CLICKHOUSE_CLIENT --query "DROP TABLE log";
}
test TinyLog | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
test StripeLog | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
test Log | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
test_func TinyLog | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
test_func StripeLog | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
test_func Log | grep -v -P '^(Memory limit|0\t0|File not found|[1-9]000000\t)'
rm "${CLICKHOUSE_TMP}/insert_result"
rm "${CLICKHOUSE_TMP}/select_result"

View File

@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# Regression for UAF in ThreadPool.
# (Triggered under TSAN)
for i in {1..10}; do
for _ in {1..10}; do
${CLICKHOUSE_LOCAL} -q 'select * from numbers_mt(100000000) settings max_threads=100 FORMAT Null'
# Binding to specific CPU is not required, but this makes the test more reliable.
taskset --cpu-list 0 ${CLICKHOUSE_LOCAL} -q 'select * from numbers_mt(100000000) settings max_threads=100 FORMAT Null'

View File

@ -16,6 +16,6 @@ opts=(
)
${CLICKHOUSE_CLIENT} "${opts[@]}" -q "SELECT groupArray(repeat('a', if(_shard_num == 2, 100000, 1))), number%100000 k from remote('127.{2,3}', system.numbers) GROUP BY k LIMIT 10e6" |& {
# the query should fail earlier on 127.3 and 127.2 should not even go to the memory limit exceeded error.
fgrep -q 'DB::Exception: Received from 127.3:9000. DB::Exception: Memory limit (for query) exceeded:'
grep -F -q 'DB::Exception: Received from 127.3:9000. DB::Exception: Memory limit (for query) exceeded:'
# while if this will not correctly then it will got the exception from the 127.2:9000 and fail
}

View File

@ -0,0 +1,3 @@
-1 DateTime64(1, \'UTC\') < 1 1 1 <= 1 1 1 = 0 0 0 >= 0 0 0 > 0 0 0 != 1 1 1
0 DateTime64(1, \'UTC\') < 0 0 0 <= 1 1 1 = 1 1 1 >= 1 1 1 > 0 0 0 != 0 0 0
1 DateTime64(1, \'UTC\') < 0 0 0 <= 0 0 0 = 0 0 0 >= 1 1 1 > 1 1 1 != 1 1 1

View File

@ -0,0 +1,43 @@
SELECT
n,
toTypeName(dt64) AS dt64_typename,
'<',
dt64 < dt,
toDateTime(dt64) < dt,
dt64 < toDateTime64(dt, 1, 'UTC'),
'<=',
dt64 <= dt,
toDateTime(dt64) <= dt,
dt64 <= toDateTime64(dt, 1, 'UTC'),
'=',
dt64 = dt,
toDateTime(dt64) = dt,
dt64 = toDateTime64(dt, 1, 'UTC'),
'>=',
dt64 >= dt,
toDateTime(dt64) >= dt,
dt64 >= toDateTime64(dt, 1, 'UTC'),
'>',
dt64 > dt,
toDateTime(dt64) > dt,
dt64 > toDateTime64(dt, 1, 'UTC'),
'!=',
dt64 != dt,
toDateTime(dt64) != dt,
dt64 != toDateTime64(dt, 1, 'UTC')
FROM
(
WITH toDateTime('2015-05-18 07:40:11') as value
SELECT
number - 1 as n,
toDateTime64(value, 1, 'UTC') AS dt64,
value - n as dt
FROM system.numbers
LIMIT 3
)

View File

@ -0,0 +1,3 @@
-1 DateTime64(1, \'UTC\') < 1 1 1 <= 1 1 1 = 0 0 0 >= 0 0 0 > 0 0 0 != 1 1 1
0 DateTime64(1, \'UTC\') < 0 0 0 <= 0 1 0 = 0 1 0 >= 1 1 1 > 1 0 1 != 1 0 1
1 DateTime64(1, \'UTC\') < 0 0 0 <= 0 0 0 = 0 0 0 >= 1 1 1 > 1 1 1 != 1 1 1

View File

@ -0,0 +1,43 @@
SELECT
n,
toTypeName(dt64) AS dt64_typename,
'<',
dt64 < d,
toDate(dt64) < d,
dt64 < toDateTime64(d, 1, 'UTC'),
'<=',
dt64 <= d,
toDate(dt64) <= d,
dt64 <= toDateTime64(d, 1, 'UTC'),
'=',
dt64 = d,
toDate(dt64) = d,
dt64 = toDateTime64(d, 1, 'UTC'),
'>=',
dt64 >= d,
toDate(dt64) >= d,
dt64 >= toDateTime64(d, 1, 'UTC'),
'>',
dt64 > d,
toDate(dt64) > d,
dt64 > toDateTime64(d, 1, 'UTC'),
'!=',
dt64 != d,
toDate(dt64) != d,
dt64 != toDateTime64(d, 1, 'UTC')
FROM
(
WITH toDateTime('2019-09-16 19:20:11') as val
SELECT
number - 1 as n,
toDateTime64(val, 1, 'UTC') AS dt64,
toDate(val, 'UTC') - n as d
FROM system.numbers
LIMIT 3
)

View File

@ -120,3 +120,7 @@ INSERT INTO partial_duplicates SELECT * FROM source_data;
OPTIMIZE TABLE partial_duplicates FINAL DEDUPLICATE BY COLUMNS('.*k');
SELECT * FROM partial_duplicates;
TRUNCATE partial_duplicates;
DROP TABLE full_duplicates;
DROP TABLE partial_duplicates;
DROP TABLE source_data;

View File

@ -0,0 +1,4 @@
['Hello','world','42" TV']
['Hello','world','42" TV']
['Hello','world','42" TV']
['Hello','world','42" TV']

View File

@ -0,0 +1,27 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --multiquery --query "
DROP TABLE IF EXISTS test;
CREATE TABLE test (a Array(String)) ENGINE = Memory;
"
${CLICKHOUSE_CLIENT} --query "INSERT INTO test FORMAT CSV" <<END
"['Hello', 'world', '42"" TV']"
END
${CLICKHOUSE_CLIENT} --format_csv_allow_single_quotes 0 --query "INSERT INTO test FORMAT CSV" <<END
"'Hello', 'world', '42"" TV'"
END
${CLICKHOUSE_CLIENT} --input_format_csv_arrays_as_nested_csv 1 --query "INSERT INTO test FORMAT CSV" <<END
"[""Hello"", ""world"", ""42"""" TV""]"
"""Hello"", ""world"", ""42"""" TV"""
END
${CLICKHOUSE_CLIENT} --multiquery --query "
SELECT * FROM test;
DROP TABLE IF EXISTS test;
"

View File

@ -0,0 +1,25 @@
#!/usr/bin/expect -f
log_user 0
set timeout 5
match_max 100000
if ![info exists env(CLICKHOUSE_PORT_TCP)] {set env(CLICKHOUSE_PORT_TCP) 9000}
set env(EDITOR) [file dirname [file normalize [info script]]]"/01610_client_spawn_editor_open.editor"
spawn clickhouse-client
expect ":) "
# Open EDITOR
send -- "\033E"
# Send return
send -- "\r"
expect {
"│ 1 │" { }
timeout { exit 1 }
}
expect ":) "
send -- ""
expect eof

View File

@ -0,0 +1,3 @@
#!/usr/bin/env bash
echo "select 1" > "$1"

View File

@ -1 +1 @@
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)

View File

@ -1 +1 @@
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)

View File

@ -1 +1 @@
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)
`id` Nullable(Int32), `bool_col` Nullable(UInt8), `tinyint_col` Nullable(Int32), `smallint_col` Nullable(Int32), `int_col` Nullable(Int32), `bigint_col` Nullable(Int64), `float_col` Nullable(Float32), `double_col` Nullable(Float64), `date_string_col` Nullable(String), `string_col` Nullable(String), `timestamp_col` Nullable(Int64)

Binary file not shown.

View File

@ -0,0 +1 @@
`foo` Nullable(String)

View File

@ -1 +1 @@
`value` Nullable(Decimal128(1))
`value` Nullable(Decimal(4, 2))

View File

@ -1 +1 @@
`a` Nullable(String), `b` Nullable(Int32), `c` Nullable(Float64), `d` Nullable(UInt8), `element` Nullable(Int32)
`a` Nullable(String), `b` Nullable(Int32), `c` Nullable(Float64), `d` Nullable(UInt8), `e` Nullable(Int32)

View File

@ -0,0 +1 @@
`l_partkey` Nullable(Int32)

View File

@ -0,0 +1 @@
`value` Nullable(Decimal(25, 2))

View File

@ -1 +1 @@
`value` Nullable(Decimal128(1))
`value` Nullable(Decimal(25, 2))

View File

@ -1 +1 @@
`value` Nullable(Decimal128(1))
`value` Nullable(Decimal(13, 2))

View File

@ -0,0 +1 @@
`c0` Nullable(Int64), `c1` Nullable(String), `v11` Nullable(Float64)

View File

@ -1 +1 @@
`value` Nullable(Decimal128(1))
`value` Nullable(Decimal(4, 2))

View File

@ -1 +1 @@
`value` Nullable(Decimal128(1))
`value` Nullable(Decimal(10, 2))

View File

@ -0,0 +1 @@
`int64_list` Nullable(Int64), `utf8_list` Nullable(String)

View File

@ -1 +1 @@
`nation_key` Nullable(Int32), `name` Nullable(String), `region_key` Nullable(Int32), `comment_col` Nullable(String)
`nation_key` Nullable(Int32), `name` Nullable(String), `region_key` Nullable(Int32), `comment_col` Nullable(String)

View File

@ -1 +1 @@
`element` Nullable(String), `b` Nullable(Int32)
`a` Nullable(String), `b` Nullable(Int32)

View File

@ -1 +1 @@
`key` Nullable(String), `key1` Nullable(Int32), `value` Nullable(UInt8), `b` Nullable(Int32), `c` Nullable(Float64)
`a` Tuple(Nullable(String), Nullable(Int32), Nullable(UInt8)), `b` Nullable(Int32), `c` Nullable(Float64)

View File

@ -0,0 +1 @@
`c0` Nullable(Int64), `c1` Nullable(String), `v11` Nullable(Float64)

View File

@ -1 +1 @@
`ID` Nullable(Int64), `element` Nullable(Int32), `element2` Nullable(Int32), `key` Nullable(String), `value` Nullable(Int32), `key5` Nullable(String), `value6` Nullable(Int32), `a` Nullable(Int32), `element8` Nullable(Int32), `e` Nullable(Int32), `f` Nullable(String), `key11` Nullable(String), `element12` Nullable(Float64)
`ID` Nullable(Int64), `Int_Array` Nullable(Int32), `int_array_array` Nullable(Int32), `Int_Map` Tuple(Nullable(String), Nullable(Int32)), `int_map_array` Tuple(Nullable(String), Nullable(Int32)), `nested_Struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64))

View File

@ -1 +1 @@
`id` Nullable(Int64), `element` Nullable(Int32), `element2` Nullable(Int32), `key` Nullable(String), `value` Nullable(Int32), `key5` Nullable(String), `value6` Nullable(Int32), `A` Nullable(Int32), `element8` Nullable(Int32), `E` Nullable(Int32), `F` Nullable(String), `key11` Nullable(String), `element12` Nullable(Float64)
`id` Nullable(Int64), `int_array` Nullable(Int32), `int_array_Array` Nullable(Int32), `int_map` Tuple(Nullable(String), Nullable(Int32)), `int_Map_Array` Tuple(Nullable(String), Nullable(Int32)), `nested_struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64))

View File

@ -1 +1 @@
`b_c_int` Nullable(Int32)
`b_struct` Nullable(Int32)

View File

@ -1 +0,0 @@
`id` Nullable(Int32), `number` Nullable(Int64), `kind` Nullable(String)

View File

@ -0,0 +1 @@
`mycol` Nullable(Float64)

View File

@ -1 +1 @@
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)

View File

@ -1 +1 @@
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)

View File

@ -1 +1 @@
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)

View File

@ -1 +1 @@
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)

View File

@ -1 +1 @@
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)
`registration_dttm` Nullable(Int64), `id` Nullable(Int32), `first_name` Nullable(String), `last_name` Nullable(String), `email` Nullable(String), `gender` Nullable(String), `ip_address` Nullable(String), `cc` Nullable(String), `country` Nullable(String), `birthdate` Nullable(String), `salary` Nullable(Float64), `title` Nullable(String), `comments` Nullable(String)

View File

@ -1 +1 @@
`carat` Nullable(Float64), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `cut` Nullable(String), `color` Nullable(String), `clarity` Nullable(String)
`carat` Nullable(Float64), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `cut` Nullable(String), `color` Nullable(String), `clarity` Nullable(String)

View File

@ -1 +1 @@
`a` Nullable(Int64), `b` Nullable(Float64), `c` Nullable(DateTime), `index` Nullable(String), `__index_level_1__` Nullable(DateTime)
`a` Nullable(Int64), `b` Nullable(Float64), `c` Nullable(DateTime), `index` Nullable(String), `__index_level_1__` Nullable(DateTime)

View File

@ -1 +1 @@
`carat` Nullable(Float64), `cut` Nullable(String), `color` Nullable(String), `clarity` Nullable(String), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `__index_level_0__` Nullable(Int64)
`carat` Nullable(Float64), `cut` Nullable(String), `color` Nullable(String), `clarity` Nullable(String), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `__index_level_0__` Nullable(Int64)

View File

@ -1 +1 @@
`carat` Nullable(Float64), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `cut` Nullable(String), `__index_level_1__` Nullable(String), `clarity` Nullable(String)
`carat` Nullable(Float64), `depth` Nullable(Float64), `table` Nullable(Float64), `price` Nullable(Int64), `x` Nullable(Float64), `y` Nullable(Float64), `z` Nullable(Float64), `cut` Nullable(String), `__index_level_1__` Nullable(String), `clarity` Nullable(String)

View File

@ -0,0 +1,88 @@
#!/usr/bin/env python3
import json
import sys
TYPE_PARQUET_CONVERTED_TO_CLICKHOUSE = {
"TIMESTAMP_MICROS": "DateTime",
"TIMESTAMP_MILLIS": "DateTime",
"UTF8": "String",
}
TYPE_PARQUET_PHYSICAL_TO_CLICKHOUSE = {
"BOOLEAN": "UInt8",
"INT32": "Int32",
"INT64": "Int64",
"FLOAT": "Float32",
"DOUBLE": "Float64",
"BYTE_ARRAY": "String",
"INT96": "Int64", # TODO!
}
def read_file(filename):
with open(filename, "rb") as f:
return f.read().decode("raw_unicode_escape")
def get_column_name(column):
return column["Name"].split(".", 1)[0]
def resolve_clickhouse_column_type(column):
column_name = get_column_name(column)
logical_type = column.get("LogicalType", {})
converted_type = column.get("ConvertedType", "").upper()
physical_type = column.get("PhysicalType", "").upper()
if logical_type and logical_type.get("Type", "").upper() == "DECIMAL":
precision = int(logical_type["precision"])
scale = int(logical_type["scale"])
if precision < 1 or precision > 76:
raise RuntimeError("Column {} has invalid Decimal precision {}".format(column_name, precision))
if precision > 38:
raise RuntimeError("Column {} has unsupported Decimal precision {}".format(column_name, precision))
if scale < 0 or scale > precision:
raise RuntimeError("Column {} has invalid Decimal scale {} for precision {}".format(column_name, scale, precision))
return "Decimal({}, {})".format(precision, scale)
if converted_type and converted_type != "NONE":
result_type = TYPE_PARQUET_CONVERTED_TO_CLICKHOUSE.get(converted_type)
if result_type:
return result_type
raise RuntimeError("Column {} has unknown ConvertedType: {}".format(column_name, converted_type))
if physical_type and physical_type != "NONE":
result_type = TYPE_PARQUET_PHYSICAL_TO_CLICKHOUSE.get(physical_type)
if result_type:
return result_type
raise RuntimeError("Column {} has unknown PhysicalType: {}".format(column_name, physical_type))
raise RuntimeError("Column {} has invalid types: ConvertedType={}, PhysicalType={}".format(column_name, converted_type, physical_type))
def dump_columns(obj):
descr_by_column_name = {}
columns_descr = []
for column in obj["Columns"]:
column_name = get_column_name(column)
column_type = resolve_clickhouse_column_type(column)
result_type = "Nullable({})".format(column_type)
if column_name in descr_by_column_name:
descr = descr_by_column_name[column_name]
descr["types"].append(result_type)
else:
descr = {
"name": column_name,
"types": [result_type],
}
descr_by_column_name[column_name] = descr
columns_descr.append(descr)
# Make tuples from nested types. CH Server doesn't support such Arrow type but it makes Server Exceptions more relevant.
def _format_type(types):
if len(types) == 1:
return types[0]
else:
return "Tuple({})".format(", ".join(types))
print(", ".join(map(lambda descr: "`{}` {}".format(descr["name"], _format_type(descr["types"])), columns_descr)))
def dump_columns_from_file(filename):
dump_columns(json.loads(read_file(filename), strict=False))
if __name__ == "__main__":
filename = sys.argv[1]
dump_columns_from_file(filename)

View File

@ -20,6 +20,7 @@ SKIP_LIST = [
"00505_shard_secure",
"00506_union_distributed", # flaky
"00646_url_engine",
"00821_distributed_storage_with_join_on.sql", # flaky
"00834_cancel_http_readonly_queries_on_client_close",
"00933_test_fix_extra_seek_on_compressed_cache",
"00965_logs_level_bugfix",
@ -31,6 +32,7 @@ SKIP_LIST = [
"01018_ip_dictionary",
"01023_materialized_view_query_context", # flaky
"01035_lc_empty_part_bug", # flaky
"01037_polygon_dicts_simple_functions.sh", # flaky
"01046_materialized_view_with_join_over_distributed", # flaky
"01050_clickhouse_dict_source_with_subquery",
"01053_ssd_dictionary",
@ -45,6 +47,7 @@ SKIP_LIST = [
"01103_check_cpu_instructions_at_startup",
"01114_database_atomic",
"01148_zookeeper_path_macros_unfolding",
"01193_metadata_loading.sh", # flaky
"01274_alter_rename_column_distributed", # flaky
"01280_ssd_complex_key_dictionary",
"01293_client_interactive_vertical_multiline", # expect-test
@ -58,6 +61,7 @@ SKIP_LIST = [
"01320_create_sync_race_condition_zookeeper",
"01355_CSV_input_format_allow_errors",
"01370_client_autocomplete_word_break_characters", # expect-test
"01375_storage_file_tsv_csv_with_names_write_prefix", # flaky
"01376_GROUP_BY_injective_elimination_dictGet",
"01393_benchmark_secure_port",
"01418_custom_settings",

View File

@ -1,143 +0,0 @@
<?xml version="1.0"?>
<!-- Config for test server -->
<yandex>
<logger>
<level>trace</level>
<log>/tmp/clickhouse/log/clickhouse-server.log</log>
<errorlog>/tmp/clickhouse/log/clickhouse-server.err.log</errorlog>
<size>10M</size>
<count>1</count>
<compress>0</compress>
</logger>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<listen_try>1</listen_try>
<http_port>58123</http_port>
<tcp_port>59000</tcp_port>
<https_port>58443</https_port>
<tcp_port_secure>59440</tcp_port_secure>
<interserver_http_port>59009</interserver_http_port>
<max_thread_pool_size>10000</max_thread_pool_size>
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
<certificateFile>/tmp/clickhouse/etc/server.crt</certificateFile>
<privateKeyFile>/tmp/clickhouse/etc/server.key</privateKeyFile>
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
<dhParamsFile>/tmp/clickhouse/etc/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
<client> <!-- Used for connecting to https dictionary source and secured Zookeeper communication -->
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<keep_alive_timeout>3</keep_alive_timeout>
<path>/tmp/clickhouse/data/</path>
<tmp_path>/tmp/clickhouse/tmp/</tmp_path>
<users_config>users.xml</users_config>
<access_control_path>/tmp/clickhouse/data/access/</access_control_path>
<custom_settings_prefixes>custom_</custom_settings_prefixes>
<mark_cache_size>5368709120</mark_cache_size>
<default_profile>default</default_profile>
<default_database>default</default_database>
<timezone>Europe/Moscow</timezone>
<remote_servers incl="clickhouse_remote_servers" >
<!-- Test only shard config for testing distributed storage -->
<test_unavailable_shard>
<shard>
<replica>
<host>localhost</host>
<port>59000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>1</port>
</replica>
</shard>
</test_unavailable_shard>
<test_shard_localhost>
<shard>
<replica>
<host>localhost</host>
<port>59000</port>
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>59000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>59000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_cluster_two_shards_localhost>
<shard>
<replica>
<host>localhost</host>
<port>59000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>59000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
<test_shard_localhost_secure>
<shard>
<replica>
<host>localhost</host>
<port>59440</port>
<secure>1</secure>
</replica>
</shard>
</test_shard_localhost_secure>
</remote_servers>
<include_from/>
<zookeeper incl="zookeeper-servers" optional="true" />
<macros incl="macros" optional="true" />
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
<max_session_timeout>3600</max_session_timeout>
<default_session_timeout>60</default_session_timeout>
<query_log>
<database>system</database>
<table>query_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<compression incl="clickhouse_compression">
</compression>
<distributed_ddl>
<path>/clickhouse/task_queue/ddl</path>
</distributed_ddl>
<format_schema_path>/tmp/clickhouse/data/format_schemas/</format_schema_path>
<query_masking_rules>
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</yandex>

View File

@ -131,4 +131,8 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
# Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep 'std::[io]\?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream"
xargs grep -P 'std::[io]?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream"
# Conflict markers
find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' |
xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files"

View File

@ -1,8 +1,13 @@
# -*- coding: utf-8 -*-
from clickhouse.utils.github.cherrypick import CherryPick
from clickhouse.utils.github.query import Query as RemoteRepo
from clickhouse.utils.github.local import Repository as LocalRepo
try:
from clickhouse.utils.github.cherrypick import CherryPick
from clickhouse.utils.github.query import Query as RemoteRepo
from clickhouse.utils.github.local import Repository as LocalRepo
except:
from .cherrypick import CherryPick
from .query import Query as RemoteRepo
from .local import Repository as LocalRepo
import argparse
import logging
@ -20,9 +25,25 @@ class Backport:
def getPullRequests(self, from_commit):
return self._gh.get_pull_requests(from_commit)
def execute(self, repo, until_commit, number, run_cherrypick):
def getBranchesWithLTS(self):
branches = []
for pull_request in self._gh.find_pull_requests("release-lts"):
if not pull_request['merged'] and not pull_request['closed']:
branches.append(pull_request['headRefName'])
return branches
def execute(self, repo, until_commit, number, run_cherrypick, find_lts=False):
repo = LocalRepo(repo, 'origin', self.default_branch_name)
branches = repo.get_release_branches()[-number:] # [(branch_name, base_commit)]
all_branches = repo.get_release_branches() # [(branch_name, base_commit)]
last_branches = set([branch[0] for branch in all_branches[-number:]])
lts_branches = set(self.getBranchesWithLTS() if find_lts else [])
branches = []
# iterate over all branches to preserve their precedence.
for branch in all_branches:
if branch in last_branches or branch in lts_branches:
branches.append(branch)
if not branches:
logging.info('No release branches found!')
@ -95,6 +116,7 @@ if __name__ == "__main__":
parser.add_argument('--repo', type=str, required=True, help='path to full repository', metavar='PATH')
parser.add_argument('--til', type=str, help='check PRs from HEAD til this commit', metavar='COMMIT')
parser.add_argument('-n', type=int, dest='number', help='number of last release branches to consider')
parser.add_argument('--lts', action='store_true', help='consider branches with LTS')
parser.add_argument('--dry-run', action='store_true', help='do not create or merge any PRs', default=False)
parser.add_argument('--verbose', '-v', action='store_true', help='more verbose output', default=False)
args = parser.parse_args()
@ -106,4 +128,4 @@ if __name__ == "__main__":
cherrypick_run = lambda token, pr, branch: CherryPick(token, 'ClickHouse', 'ClickHouse', 'core', pr, branch).execute(args.repo, args.dry_run)
bp = Backport(args.token, 'ClickHouse', 'ClickHouse', 'core')
bp.execute(args.repo, args.til, args.number, cherrypick_run)
bp.execute(args.repo, args.til, args.number, cherrypick_run, args.lts)

View File

@ -14,7 +14,10 @@ Second run checks PR from previous run to be merged or at least being mergeable.
Third run creates PR from backport branch (with merged previous PR) to release branch.
'''
from clickhouse.utils.github.query import Query as RemoteRepo
try:
from clickhouse.utils.github.query import Query as RemoteRepo
except:
from .query import Query as RemoteRepo
import argparse
from enum import Enum

View File

@ -39,6 +39,7 @@ class Query:
baseRefName
closed
headRefName
id
mergeable
merged
@ -158,6 +159,24 @@ class Query:
else:
return {}
def find_pull_requests(self, label_name):
'''
Get all pull-requests filtered by label name
'''
_QUERY = '''
repository(owner: "{owner}" name: "{name}") {{
pullRequests(first: {min_page_size} labels: "{label_name}") {{
nodes {{
{pull_request_data}
}}
}}
}}
'''
query = _QUERY.format(owner=self._owner, name=self._name, label_name=label_name,
pull_request_data=self._PULL_REQUEST, min_page_size=self._min_page_size)
return self._run(query)['repository']['pullRequests']['nodes']
def get_pull_requests(self, before_commit):
'''
Get all merged pull-requests from the HEAD of default branch to the last commit (excluding)
@ -342,130 +361,6 @@ class Query:
query = _SET_LABEL.format(pr_id=pull_request['id'], label_id=labels[0]['id'])
self._run(query, is_mutation=True)
# OLD METHODS
# _LABELS = '''
# repository(owner: "ClickHouse" name: "ClickHouse") {{
# pullRequest(number: {number}) {{
# labels(first: {max_page_size} {next}) {{
# pageInfo {{
# hasNextPage
# endCursor
# }}
# nodes {{
# name
# color
# }}
# }}
# }}
# }}
# '''
# def get_labels(self, pull_request):
# '''Fetchs all labels for given pull-request
# Args:
# pull_request: JSON object returned by `get_pull_requests()`
# Returns:
# labels: a list of JSON nodes with the name and color fields
# '''
# labels = [label for label in pull_request['labels']['nodes']]
# not_end = pull_request['labels']['pageInfo']['hasNextPage']
# query = Query._LABELS.format(number = pull_request['number'],
# max_page_size = self._max_page_size,
# next=f'after: "{pull_request["labels"]["pageInfo"]["endCursor"]}"')
# while not_end:
# result = self._run(query)['repository']['pullRequest']['labels']
# not_end = result['pageInfo']['hasNextPage']
# query = Query._LABELS.format(number=pull_request['number'],
# max_page_size=self._max_page_size,
# next=f'after: "{result["pageInfo"]["endCursor"]}"')
# labels += [label for label in result['nodes']]
# return labels
# _TIMELINE = '''
# repository(owner: "ClickHouse" name: "ClickHouse") {{
# pullRequest(number: {number}) {{
# timeline(first: {max_page_size} {next}) {{
# pageInfo {{
# hasNextPage
# endCursor
# }}
# nodes {{
# ... on CrossReferencedEvent {{
# isCrossRepository
# source {{
# ... on PullRequest {{
# number
# baseRefName
# merged
# labels(first: {max_page_size}) {{
# pageInfo {{
# hasNextPage
# endCursor
# }}
# nodes {{
# name
# color
# }}
# }}
# }}
# }}
# target {{
# ... on PullRequest {{
# number
# }}
# }}
# }}
# }}
# }}
# }}
# }}
# '''
# def get_timeline(self, pull_request):
# '''Fetchs all cross-reference events from pull-request's timeline
# Args:
# pull_request: JSON object returned by `get_pull_requests()`
# Returns:
# events: a list of JSON nodes for CrossReferenceEvent
# '''
# events = [event for event in pull_request['timeline']['nodes'] if event and event['source']]
# not_end = pull_request['timeline']['pageInfo']['hasNextPage']
# query = Query._TIMELINE.format(number = pull_request['number'],
# max_page_size = self._max_page_size,
# next=f'after: "{pull_request["timeline"]["pageInfo"]["endCursor"]}"')
# while not_end:
# result = self._run(query)['repository']['pullRequest']['timeline']
# not_end = result['pageInfo']['hasNextPage']
# query = Query._TIMELINE.format(number=pull_request['number'],
# max_page_size=self._max_page_size,
# next=f'after: "{result["pageInfo"]["endCursor"]}"')
# events += [event for event in result['nodes'] if event and event['source']]
# return events
# _DEFAULT = '''
# repository(owner: "ClickHouse", name: "ClickHouse") {
# defaultBranchRef {
# name
# }
# }
# '''
# def get_default_branch(self):
# '''Get short name of the default branch
# Returns:
# name (string): branch name
# '''
# return self._run(Query._DEFAULT)['repository']['defaultBranchRef']['name']
def _run(self, query, is_mutation=False):
from requests.adapters import HTTPAdapter
from urllib3.util.retry import Retry