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

This commit is contained in:
zhang2014 2020-08-15 16:12:46 +08:00
commit fe49bd3414
73 changed files with 533 additions and 520 deletions

View File

@ -368,7 +368,6 @@ include (cmake/find/icu.cmake)
include (cmake/find/zlib.cmake)
include (cmake/find/zstd.cmake)
include (cmake/find/ltdl.cmake) # for odbc
include (cmake/find/termcap.cmake)
# openssl, zlib before poco
include (cmake/find/sparsehash.cmake)
include (cmake/find/re2.cmake)

View File

@ -1,6 +1,10 @@
option(ENABLE_CASSANDRA "Enable Cassandra" ${ENABLE_LIBRARIES})
if (ENABLE_CASSANDRA)
if (APPLE)
SET(CMAKE_MACOSX_RPATH ON)
endif()
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv")
message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive")
elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra")

View File

@ -1,8 +0,0 @@
find_library (TERMCAP_LIBRARY tinfo)
if (NOT TERMCAP_LIBRARY)
find_library (TERMCAP_LIBRARY ncurses)
endif()
if (NOT TERMCAP_LIBRARY)
find_library (TERMCAP_LIBRARY termcap)
endif()
message (STATUS "Using termcap: ${TERMCAP_LIBRARY}")

View File

@ -122,11 +122,11 @@ initdb()
CLICKHOUSE_DATADIR_FROM_CONFIG=$CLICKHOUSE_DATADIR
fi
if ! getent group ${CLICKHOUSE_USER} >/dev/null; then
if ! getent passwd ${CLICKHOUSE_USER} >/dev/null; then
echo "Can't chown to non-existing user ${CLICKHOUSE_USER}"
return
fi
if ! getent passwd ${CLICKHOUSE_GROUP} >/dev/null; then
if ! getent group ${CLICKHOUSE_GROUP} >/dev/null; then
echo "Can't chown to non-existing group ${CLICKHOUSE_GROUP}"
return
fi

View File

@ -2,6 +2,7 @@
"docker/packager/deb": {
"name": "yandex/clickhouse-deb-builder",
"dependent": [
"docker/packager/unbundled",
"docker/test/stateless",
"docker/test/stateless_with_coverage",
"docker/test/stateless_pytest",
@ -15,6 +16,10 @@
"docker/test/pvs"
]
},
"docker/packager/unbundled": {
"name": "yandex/clickhouse-unbundled-builder",
"dependent": []
},
"docker/test/coverage": {
"name": "yandex/clickhouse-coverage",
"dependent": []
@ -92,6 +97,10 @@
"name": "yandex/clickhouse-fasttest",
"dependent": []
},
"docker/test/style": {
"name": "yandex/clickhouse-style-test",
"dependent": []
},
"docker/test/integration/s3_proxy": {
"name": "yandex/clickhouse-s3-proxy",
"dependent": []

View File

@ -1,9 +1,9 @@
# docker build -t yandex/clickhouse-deb-builder .
FROM ubuntu:19.10
FROM ubuntu:20.04
RUN apt-get --allow-unauthenticated update -y && apt-get install --yes wget gnupg
RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add -
RUN echo "deb [trusted=yes] http://apt.llvm.org/eoan/ llvm-toolchain-eoan-10 main" >> /etc/apt/sources.list
RUN echo "deb [trusted=yes] http://apt.llvm.org/focal/ llvm-toolchain-focal-11 main" >> /etc/apt/sources.list
# initial packages
RUN apt-get --allow-unauthenticated update -y \
@ -25,13 +25,17 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb
RUN chmod +x dpkg-deb
RUN cp dpkg-deb /usr/bin
# Libraries from OS are only needed to test the "unbundled" build (that is not used in production).
RUN apt-get --allow-unauthenticated update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get --allow-unauthenticated install --yes --no-install-recommends \
gcc-10 \
g++-10 \
gcc-9 \
g++-9 \
llvm-11 \
clang-11 \
lld-11 \
clang-tidy-11 \
llvm-10 \
clang-10 \
lld-10 \
@ -39,54 +43,19 @@ RUN apt-get --allow-unauthenticated update -y \
clang-9 \
lld-9 \
clang-tidy-9 \
libicu-dev \
libreadline-dev \
gperf \
ninja-build \
perl \
pkg-config \
devscripts \
debhelper \
git \
libc++-dev \
libc++abi-dev \
libboost-program-options-dev \
libboost-system-dev \
libboost-filesystem-dev \
libboost-thread-dev \
libboost-iostreams-dev \
libboost-regex-dev \
zlib1g-dev \
liblz4-dev \
libdouble-conversion-dev \
librdkafka-dev \
libpoconetssl62 \
libpoco-dev \
libgoogle-perftools-dev \
libzstd-dev \
libltdl-dev \
libre2-dev \
libjemalloc-dev \
libmsgpack-dev \
libcurl4-openssl-dev \
opencl-headers \
ocl-icd-libopencl1 \
intel-opencl-icd \
unixodbc-dev \
odbcinst \
tzdata \
gperf \
alien \
libcapnp-dev \
cmake \
gdb \
pigz \
moreutils \
libcctz-dev \
libldap2-dev \
libsasl2-dev \
heimdal-multidev \
libhyperscan-dev
pigz
# This symlink required by gcc to find lld compiler

View File

@ -11,6 +11,7 @@ SCRIPT_PATH = os.path.realpath(__file__)
IMAGE_MAP = {
"deb": "yandex/clickhouse-deb-builder",
"binary": "yandex/clickhouse-binary-builder",
"unbundled": "yandex/clickhouse-unbundled-builder"
}
def check_image_exists_locally(image_name):
@ -176,7 +177,9 @@ if __name__ == "__main__":
parser.add_argument("--clickhouse-repo-path", default=os.path.join(os.path.dirname(os.path.abspath(__file__)), os.pardir, os.pardir))
parser.add_argument("--output-dir", required=True)
parser.add_argument("--build-type", choices=("debug", ""), default="")
parser.add_argument("--compiler", choices=("clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", "gcc-9", "clang-10"), default="gcc-9")
parser.add_argument("--compiler", choices=("clang-10", "clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd",
"clang-11", "clang-11-darwin", "clang-11-aarch64", "clang-11-freebsd",
"gcc-9", "gcc-10"), default="gcc-9")
parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="")
parser.add_argument("--unbundled", action="store_true")
parser.add_argument("--split-binary", action="store_true")
@ -197,7 +200,7 @@ if __name__ == "__main__":
if not os.path.isabs(args.output_dir):
args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir))
image_type = 'binary' if args.package_type == 'performance' else args.package_type
image_type = 'binary' if args.package_type == 'performance' else 'unbundled' if args.unbundled else args.package_type
image_name = IMAGE_MAP[image_type]
if not os.path.isabs(args.clickhouse_repo_path):

View File

@ -0,0 +1,56 @@
# docker build -t yandex/clickhouse-unbundled-builder .
FROM yandex/clickhouse-deb-builder
# Libraries from OS are only needed to test the "unbundled" build (that is not used in production).
RUN apt-get --allow-unauthenticated update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get --allow-unauthenticated install --yes --no-install-recommends \
libicu-dev \
libreadline-dev \
gperf \
perl \
pkg-config \
devscripts \
libc++-dev \
libc++abi-dev \
libboost-program-options-dev \
libboost-system-dev \
libboost-filesystem-dev \
libboost-thread-dev \
libboost-iostreams-dev \
libboost-regex-dev \
zlib1g-dev \
liblz4-dev \
libdouble-conversion-dev \
librdkafka-dev \
libpoconetssl62 \
libpoco-dev \
libgoogle-perftools-dev \
libzstd-dev \
libltdl-dev \
libre2-dev \
libjemalloc-dev \
libmsgpack-dev \
libcurl4-openssl-dev \
opencl-headers \
ocl-icd-libopencl1 \
intel-opencl-icd \
unixodbc-dev \
odbcinst \
tzdata \
gperf \
alien \
libcapnp-dev \
cmake \
gdb \
pigz \
moreutils \
libcctz-dev \
libldap2-dev \
libsasl2-dev \
heimdal-multidev \
libhyperscan-dev
COPY build.sh /
CMD ["/bin/bash", "/build.sh"]

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
set -x -e
# Update tzdata to the latest version. It is embedded into clickhouse binary.
sudo apt-get update && sudo apt-get install tzdata
ccache --show-stats ||:
ccache --zero-stats ||:
build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S'
mv /*.deb /output
mv *.changes /output
mv *.buildinfo /output
mv /*.rpm /output ||: # if exists
mv /*.tgz /output ||: # if exists
ccache --show-stats ||:
ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||:

View File

@ -24,20 +24,26 @@ def run_perf_test(cmd, xmls_path, output_folder):
return p
def get_options(i):
options = ""
if 0 < i:
options += " --order=random"
if i == 1:
options += " --atomic-db-engine"
return options
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option):
skip_list_opt = get_skip_list_cmd(cmd)
output_paths = [os.path.join(output_prefix, "stress_test_run_{}.txt".format(i)) for i in range(num_processes)]
f = open(output_paths[0], 'w')
main_command = "{} {} {}".format(cmd, skip_list_opt, skip_tests_option)
logging.info("Run func tests main cmd '%s'", main_command)
pipes = [Popen(main_command, shell=True, stdout=f, stderr=f)]
for output_path in output_paths[1:]:
time.sleep(0.5)
f = open(output_path, 'w')
full_command = "{} {} --order=random {}".format(cmd, skip_list_opt, skip_tests_option)
pipes = []
for i in range(0, len(output_paths)):
f = open(output_paths[i], 'w')
full_command = "{} {} {} {}".format(cmd, skip_list_opt, get_options(i), skip_tests_option)
logging.info("Run func tests '%s'", full_command)
p = Popen(full_command, shell=True, stdout=f, stderr=f)
pipes.append(p)
time.sleep(0.5)
return pipes

View File

@ -0,0 +1,8 @@
# docker build -t yandex/clickhouse-style-test .
FROM ubuntu:20.04
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git
CMD cd /ClickHouse/utils/check-style && ./check-style -n | tee /test_output/style_output.txt && \
./check-duplicate-includes.sh | tee /test_output/duplicate_output.txt

View File

@ -65,6 +65,7 @@ toc_title: Adopters
| <a href="https://rambler.ru" class="favicon">Rambler</a> | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) |
| <a href="https://www.s7.ru" class="favicon">S7 Airlines</a> | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) |
| <a href="https://www.scireum.de/" class="favicon">scireum GmbH</a> | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) |
| <a href="https://segment.com/" class="favicon">Segment</a> | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) |
| <a href="https://www.semrush.com/" class="favicon">SEMrush</a> | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) |
| <a href="https://sentry.io/" class="favicon">Sentry</a> | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) |
| <a href="https://seo.do/" class="favicon">seo.do</a> | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) |

View File

@ -399,7 +399,7 @@ The cache is shared for the server and memory is allocated as needed. The cache
```
## max\_server\_memory\_usage {#max_server_memory_usage}
Limits total RAM usage by the ClickHouse server. You can specify it only for the default profile.
Limits total RAM usage by the ClickHouse server.
Possible values:

View File

@ -1,3 +1,5 @@
# Инструкция для разработчиков
Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X.
# Если вы используете Windows {#esli-vy-ispolzuete-windows}

View File

@ -374,7 +374,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat
## max_server_memory_usage {#max_server_memory_usage}
Ограничивает объём оперативной памяти, используемой сервером ClickHouse. Настройка может быть задана только для профиля `default`.
Ограничивает объём оперативной памяти, используемой сервером ClickHouse.
Возможные значения:

View File

@ -247,12 +247,15 @@ try
context->setCurrentDatabase(default_database);
applyCmdOptions();
if (!context->getPath().empty())
String path = context->getPath();
if (!path.empty())
{
/// Lock path directory before read
status.emplace(context->getPath() + "status", StatusFile::write_full_info);
LOG_DEBUG(log, "Loading metadata from {}", context->getPath());
LOG_DEBUG(log, "Loading metadata from {}", path);
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
loadMetadataSystem(*context);
attachSystemTables(*context);
loadMetadata(*context);

View File

@ -468,6 +468,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
{
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
/// Directory with metadata of tables, which was marked as dropped by Atomic database
Poco::File(path + "metadata_dropped/").createDirectories();
}

View File

@ -388,13 +388,14 @@ public:
{
for (size_t j = 0; j < UNROLL_COUNT; ++j)
{
if (has_data[j * 256 + k])
size_t idx = j * 256 + k;
if (has_data[idx])
{
AggregateDataPtr & place = map[k];
if (unlikely(!place))
init(place);
func.merge(place + place_offset, reinterpret_cast<const char *>(&places[256 * j + k]), arena);
func.merge(place + place_offset, reinterpret_cast<const char *>(&places[idx]), nullptr);
}
}
}

View File

@ -1,229 +0,0 @@
#include <Common/typeid_cast.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/StringUtils/StringUtils.h>
#include <iostream>
#include <chrono>
#include <gtest/gtest.h>
#include <Common/ShellCommand.h>
using namespace DB;
template <typename... Args>
auto getZooKeeper(Args &&... args)
{
/// In our CI infrastructure it is typical that ZooKeeper is unavailable for some amount of time.
size_t i;
for (i = 0; i < 100; ++i)
{
try
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", std::forward<Args>(args)...);
zookeeper->exists("/");
zookeeper->createIfNotExists("/clickhouse_test", "Unit tests of ClickHouse");
return zookeeper;
}
catch (...)
{
std::cerr << "Zookeeper is unavailable, try " << i << std::endl;
sleep(1);
continue;
}
}
std::cerr << "No zookeeper after " << i << " tries. skip tests." << std::endl;
exit(0);
}
TEST(zkutil, MultiNiceExceptionMsg)
{
auto zookeeper = getZooKeeper();
Coordination::Requests ops;
ASSERT_NO_THROW(
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
);
try
{
ops.clear();
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/c", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi/c", -1));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "BadBoy", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/b", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
FAIL();
}
catch (...)
{
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
String msg = getCurrentExceptionMessage(false);
bool msg_has_reqired_patterns = msg.find("#2") != std::string::npos;
EXPECT_TRUE(msg_has_reqired_patterns) << msg;
}
}
TEST(zkutil, MultiAsync)
{
Coordination::Requests ops;
getZooKeeper()->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
{
ops.clear();
auto zookeeper = getZooKeeper();
auto fut = zookeeper->asyncMulti(ops);
}
{
ops.clear();
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "", zkutil::CreateMode::Persistent));
auto zookeeper = getZooKeeper();
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
auto res = fut.get();
ASSERT_EQ(res.error, Coordination::Error::ZOK);
ASSERT_EQ(res.responses.size(), 2);
}
EXPECT_ANY_THROW
(
auto zookeeper = getZooKeeper();
std::vector<std::future<Coordination::MultiResponse>> futures;
for (size_t i = 0; i < 10000; ++i)
{
ops.clear();
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(zkutil::makeSetRequest("/clickhouse_test/zkutil_multi", "xxx", 42));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
futures.emplace_back(zookeeper->asyncMulti(ops));
}
futures[0].get();
);
/// Check there are no segfaults for remaining 999 futures
using namespace std::chrono_literals;
std::this_thread::sleep_for(1s);
try
{
ops.clear();
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
auto zookeeper = getZooKeeper();
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
auto res = fut.get();
/// The test is quite heavy. It is normal if session is expired during this test.
/// If we don't check that, the test will be flacky.
if (res.error != Coordination::Error::ZSESSIONEXPIRED && res.error != Coordination::Error::ZCONNECTIONLOSS)
{
ASSERT_EQ(res.error, Coordination::Error::ZNODEEXISTS);
ASSERT_EQ(res.responses.size(), 2);
}
}
catch (const Coordination::Exception & e)
{
if (e.code != Coordination::Error::ZSESSIONEXPIRED && e.code != Coordination::Error::ZCONNECTIONLOSS)
throw;
}
}
TEST(zkutil, WatchGetChildrenWithChroot)
{
try
{
const String prefix = "/clickhouse_test/zkutil/watch_get_children_with_chroot";
/// Create chroot node firstly
auto zookeeper = getZooKeeper();
zookeeper->createAncestors(prefix + "/");
zookeeper = getZooKeeper("",
zkutil::DEFAULT_SESSION_TIMEOUT,
zkutil::DEFAULT_OPERATION_TIMEOUT,
prefix);
String queue_path = "/queue";
zookeeper->tryRemoveRecursive(queue_path);
zookeeper->createAncestors(queue_path + "/");
zkutil::EventPtr event = std::make_shared<Poco::Event>();
zookeeper->getChildren(queue_path, nullptr, event);
{
auto zookeeper2 = getZooKeeper("",
zkutil::DEFAULT_SESSION_TIMEOUT,
zkutil::DEFAULT_OPERATION_TIMEOUT,
prefix);
zookeeper2->create(queue_path + "/children-", "", zkutil::CreateMode::PersistentSequential);
}
event->wait();
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(true);
throw;
}
}
TEST(zkutil, MultiCreateSequential)
{
try
{
const String prefix = "/clickhouse_test/zkutil";
/// Create chroot node firstly
auto zookeeper = getZooKeeper();
zookeeper->createAncestors(prefix + "/");
zookeeper = getZooKeeper("",
zkutil::DEFAULT_SESSION_TIMEOUT,
zkutil::DEFAULT_OPERATION_TIMEOUT,
"/clickhouse_test");
String base_path = "/multi_create_sequential";
zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/");
Coordination::Requests ops;
String sequential_node_prefix = base_path + "/queue-";
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
auto results = zookeeper->multi(ops);
const auto & sequential_node_result_op = dynamic_cast<const Coordination::CreateResponse &>(*results.at(0));
EXPECT_FALSE(sequential_node_result_op.path_created.empty());
EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());
EXPECT_EQ(sequential_node_result_op.path_created.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(false);
throw;
}
}

View File

@ -63,7 +63,7 @@ namespace MySQLReplication
out << "Binlog Version: " << this->binlog_version << std::endl;
out << "Server Version: " << this->server_version << std::endl;
out << "Create Timestamp: " << this->create_timestamp << std::endl;
out << "Event Header Len: " << this->event_header_length << std::endl;
out << "Event Header Len: " << std::to_string(this->event_header_length) << std::endl;
}
/// https://dev.mysql.com/doc/internals/en/rotate-event.html
@ -123,7 +123,7 @@ namespace MySQLReplication
header.dump(out);
out << "Thread ID: " << this->thread_id << std::endl;
out << "Execution Time: " << this->exec_time << std::endl;
out << "Schema Len: " << this->schema_len << std::endl;
out << "Schema Len: " << std::to_string(this->schema_len) << std::endl;
out << "Error Code: " << this->error_code << std::endl;
out << "Status Len: " << this->status_len << std::endl;
out << "Schema: " << this->schema << std::endl;
@ -243,14 +243,14 @@ namespace MySQLReplication
header.dump(out);
out << "Table ID: " << this->table_id << std::endl;
out << "Flags: " << this->flags << std::endl;
out << "Schema Len: " << this->schema_len << std::endl;
out << "Schema Len: " << std::to_string(this->schema_len) << std::endl;
out << "Schema: " << this->schema << std::endl;
out << "Table Len: " << this->table_len << std::endl;
out << "Table Len: " << std::to_string(this->table_len) << std::endl;
out << "Table: " << this->table << std::endl;
out << "Column Count: " << this->column_count << std::endl;
for (auto i = 0U; i < column_count; i++)
{
out << "Column Type [" << i << "]: " << column_type[i] << ", Meta: " << column_meta[i] << std::endl;
out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << std::endl;
}
out << "Null Bitmap: " << this->null_bitmap << std::endl;
}

View File

@ -347,7 +347,6 @@ class IColumn;
M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
M(Bool, allow_experimental_database_atomic, true, "Allow to create database with Engine=Atomic.", 0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
@ -395,7 +394,8 @@ class IColumn;
M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
\
M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \
M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0)
M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \
M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0)
#define FORMAT_FACTORY_SETTINGS(M) \
M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \

View File

@ -366,6 +366,8 @@ void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore
std::lock_guard lock{mutex};
table_names = table_name_to_path;
}
Poco::File(path_to_table_symlinks).createDirectories();
for (const auto & table : table_names)
tryCreateSymlink(table.first, table.second);
}

View File

@ -12,11 +12,15 @@
namespace DB::ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TOO_LARGE_ARRAY_SIZE;
}
namespace DB::GatherUtils
{
inline constexpr size_t MAX_ARRAY_SIZE = 1 << 30;
/// Methods to copy Slice to Sink, overloaded for various combinations of types.
template <typename T>
@ -673,6 +677,10 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
if (size >= 0)
{
auto length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
if (array_size <= length)
{
writeSlice(array_source.getWhole(), sink);
@ -685,6 +693,10 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
else
{
auto length = static_cast<size_t>(-size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
if (array_size <= length)
{
for (size_t i = array_size; i < length; ++i)
@ -714,6 +726,10 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
if (size >= 0)
{
auto length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
if (array_size <= length)
{
writeSlice(array_source.getWhole(), sink);
@ -726,6 +742,10 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
else
{
auto length = static_cast<size_t>(-size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
if (array_size <= length)
{
for (size_t i = array_size; i < length; ++i)

View File

@ -57,7 +57,6 @@ void sliceHas(IArraySource & first, IArraySource & second, ArraySearchType searc
void push(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, bool push_front);
void resizeDynamicSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, const IColumn & size_column);
void resizeConstantSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, ssize_t size);
}

View File

@ -449,7 +449,6 @@ void NO_INLINE Aggregator::executeImpl(
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if (!no_more_keys)
//executeImplCase<false>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
executeImplBatch(method, state, aggregates_pool, rows, aggregate_instructions);
else
executeImplCase<true>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
@ -534,22 +533,36 @@ void NO_INLINE Aggregator::executeImplBatch(
/// Optimization for special case when aggregating by 8bit key.
if constexpr (std::is_same_v<Method, typename decltype(AggregatedDataVariants::key8)::element_type>)
{
/// We use another method if there are aggregate functions with -Array combinator.
bool has_arrays = false;
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
inst->batch_that->addBatchLookupTable8(
rows,
reinterpret_cast<AggregateDataPtr *>(method.data.data()),
inst->state_offset,
[&](AggregateDataPtr & aggregate_data)
{
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
},
state.getKeyData(),
inst->batch_arguments,
aggregates_pool);
if (inst->offsets)
{
has_arrays = true;
break;
}
}
if (!has_arrays)
{
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
inst->batch_that->addBatchLookupTable8(
rows,
reinterpret_cast<AggregateDataPtr *>(method.data.data()),
inst->state_offset,
[&](AggregateDataPtr & aggregate_data)
{
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
},
state.getKeyData(),
inst->batch_arguments,
aggregates_pool);
}
return;
}
return;
}
/// Generic case.
@ -629,7 +642,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl(
void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns,
AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder)
AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder)
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i].resize(params.aggregates[i].arguments.size());

View File

@ -142,10 +142,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (create.storage->engine->name == "Atomic")
{
if (!context.getSettingsRef().allow_experimental_database_atomic && !internal)
throw Exception("Atomic is an experimental database engine. "
"Enable allow_experimental_database_atomic to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY);
else if (create.uuid == UUIDHelpers::Nil)

View File

@ -28,6 +28,7 @@ namespace ErrorCodes
{
extern const int ALIAS_REQUIRED;
extern const int AMBIGUOUS_COLUMN_NAME;
extern const int LOGICAL_ERROR;
}
namespace
@ -187,7 +188,8 @@ StoragePtr JoinedTables::getLeftTableStorage()
bool JoinedTables::resolveTables()
{
tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context);
assert(tables_with_columns.size() == table_expressions.size());
if (tables_with_columns.size() != table_expressions.size())
throw Exception("Unexpected tables count", ErrorCodes::LOGICAL_ERROR);
const auto & settings = context.getSettingsRef();
if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1)

View File

@ -344,14 +344,14 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
const auto & create_materialized_column_declaration = [&](const String & name, const String & type, const auto & default_value)
{
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = name;
column_declaration->type = makeASTFunction(type);
column_declaration->default_specifier = "MATERIALIZED";
column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value);
column_declaration->children.emplace_back(column_declaration->type);
column_declaration->children.emplace_back(column_declaration->default_expression);
return std::move(column_declaration);
return column_declaration;
};
/// Add _sign and _version column.

View File

@ -15,6 +15,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
@ -111,6 +112,10 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_e
{
bool is_rewrite_tables = false;
if (tables_element.size() != tables_predicates.size())
throw Exception("Unexpected elements count in predicate push down: `set enable_optimize_predicate_expression = 0` to disable",
ErrorCodes::LOGICAL_ERROR);
for (size_t index = tables_element.size(); index > 0; --index)
{
size_t table_pos = index - 1;

View File

@ -84,9 +84,6 @@ static void loadDatabase(
}
#define SYSTEM_DATABASE "system"
void loadMetadata(Context & context, const String & default_database_name)
{
Poco::Logger * log = &Poco::Logger::get("loadMetadata");
@ -114,7 +111,7 @@ void loadMetadata(Context & context, const String & default_database_name)
if (endsWith(it.name(), ".sql"))
{
String db_name = it.name().substr(0, it.name().size() - 4);
if (db_name != SYSTEM_DATABASE)
if (db_name != DatabaseCatalog::SYSTEM_DATABASE)
databases.emplace(unescapeForFileName(db_name), path + "/" + db_name);
}
@ -140,7 +137,7 @@ void loadMetadata(Context & context, const String & default_database_name)
if (it.name().at(0) == '.')
continue;
if (it.name() == SYSTEM_DATABASE)
if (it.name() == DatabaseCatalog::SYSTEM_DATABASE)
continue;
databases.emplace(unescapeForFileName(it.name()), it.path().toString());
@ -172,21 +169,20 @@ void loadMetadata(Context & context, const String & default_database_name)
void loadMetadataSystem(Context & context)
{
String path = context.getPath() + "metadata/" SYSTEM_DATABASE;
if (Poco::File(path).exists())
String path = context.getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE;
String metadata_file = path + ".sql";
if (Poco::File(path).exists() || Poco::File(metadata_file).exists())
{
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
loadDatabase(context, SYSTEM_DATABASE, path, true);
loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true);
}
else
{
/// Initialize system database manually
String global_path = context.getPath();
Poco::File(global_path + "data/" SYSTEM_DATABASE).createDirectories();
Poco::File(global_path + "metadata/" SYSTEM_DATABASE).createDirectories();
auto system_database = std::make_shared<DatabaseOrdinary>(SYSTEM_DATABASE, global_path + "metadata/" SYSTEM_DATABASE "/", context);
DatabaseCatalog::instance().attachDatabase(SYSTEM_DATABASE, system_database);
String database_create_query = "CREATE DATABASE ";
database_create_query += DatabaseCatalog::SYSTEM_DATABASE;
database_create_query += " ENGINE=Atomic";
executeCreateQuery(database_create_query, context, DatabaseCatalog::SYSTEM_DATABASE, "<no file>", true);
}
}

View File

@ -1252,7 +1252,17 @@ void MergeTreeData::dropAllData()
clearPartsFromFilesystem(all_parts);
for (const auto & [path, disk] : getRelativeDataPathsWithDisks())
disk->removeRecursive(path);
{
try
{
disk->removeRecursive(path);
}
catch (const Poco::FileNotFoundException &)
{
/// If the file is already deleted, log the error message and do nothing.
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
LOG_TRACE(log, "dropAllData: done.");
}

View File

@ -85,7 +85,10 @@ Pipe StorageMaterializeMySQL::read(
auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList());
ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true);
pipe.addTransform(std::make_shared<FilterTransform>(pipe.getHeader(), expression_actions, filter_column_name, false));
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(header, expression_actions, filter_column_name, false);
});
}
return pipe;

View File

@ -37,6 +37,27 @@ namespace ErrorCodes
extern const int SAMPLING_NOT_SUPPORTED;
}
namespace
{
/// Rewrite original query removing joined tables from it
void removeJoin(const ASTSelectQuery & select)
{
const auto & tables = select.tables();
if (!tables || tables->children.size() < 2)
return;
const auto & joined_table = tables->children[1]->as<ASTTablesInSelectQueryElement &>();
if (!joined_table.table_join)
return;
/// The most simple temporary solution: leave only the first table in query.
/// TODO: we also need to remove joined columns and related functions (taking in account aliases if any).
tables->children.resize(1);
}
}
StorageMerge::StorageMerge(
const StorageID & table_id_,
@ -243,6 +264,9 @@ Pipe StorageMerge::createSources(
SelectQueryInfo modified_query_info = query_info;
modified_query_info.query = query_info.query->clone();
/// Original query could contain JOIN but we need only the first joined table and its columns.
removeJoin(*modified_query_info.query->as<ASTSelectQuery>());
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name);
Pipe pipe;
@ -428,9 +452,14 @@ Block StorageMerge::getQueryHeader(
}
case QueryProcessingStage::WithMergeableState:
case QueryProcessingStage::Complete:
return InterpreterSelectQuery(
query_info.query, context, std::make_shared<OneBlockInputStream>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())),
SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
{
auto query = query_info.query->clone();
removeJoin(*query->as<ASTSelectQuery>());
auto stream = std::make_shared<OneBlockInputStream>(
metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()));
return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
}
}
throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -21,7 +21,11 @@ protected:
virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0;
public:
IStorageSystemOneBlock(const String & name_) : IStorage({"system", name_})
#if defined(ARCADIA_BUILD)
IStorageSystemOneBlock(const String & name_) : IStorageSystemOneBlock(StorageID{"system", name_}) {}
#endif
IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_)
{
StorageInMemoryMetadata metadata_;
metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes()));

View File

@ -16,8 +16,8 @@ NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes()
}
StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_)
: IStorageSystemOneBlock(name_), async_metrics(async_metrics_)
StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_)
: IStorageSystemOneBlock(table_id_), async_metrics(async_metrics_)
{
}

View File

@ -25,7 +25,13 @@ private:
const AsynchronousMetrics & async_metrics;
protected:
StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_);
#if defined(ARCADIA_BUILD)
StorageSystemAsynchronousMetrics(const String & name_, const AsynchronousMetrics & async_metrics_)
: StorageSystemAsynchronousMetrics(StorageID{"system", name_}, async_metrics_)
{
}
#endif
StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_);
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
};

View File

@ -23,8 +23,8 @@ namespace ErrorCodes
extern const int TABLE_IS_DROPPED;
}
StorageSystemColumns::StorageSystemColumns(const std::string & name_)
: IStorage({"system", name_})
StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(

View File

@ -27,7 +27,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemColumns(const std::string & name_);
StorageSystemColumns(const StorageID & table_id_);
};
}

View File

@ -12,83 +12,61 @@
namespace DB
{
/**
* Implements system table 'detached_parts' which allows to get information
* about detached data parts for tables of MergeTree family.
* We don't use StorageSystemPartsBase, because it introduces virtual _state
* column and column aliases which we don't need.
*/
class StorageSystemDetachedParts final :
public ext::shared_ptr_helper<StorageSystemDetachedParts>,
public IStorage
StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_id_)
: IStorage(table_id_)
{
friend struct ext::shared_ptr_helper<StorageSystemDetachedParts>;
public:
std::string getName() const override { return "SystemDetachedParts"; }
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription{{
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"name", std::make_shared<DataTypeString>()},
{"disk", std::make_shared<DataTypeString>()},
{"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
{"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
{"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt32>())}
}});
setInMemoryMetadata(storage_metadata);
}
protected:
explicit StorageSystemDetachedParts()
: IStorage({"system", "detached_parts"})
Pipe StorageSystemDetachedParts::read(
const Names & /* column_names */,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
StoragesInfoStream stream(query_info, context);
/// Create the result.
Block block = metadata_snapshot->getSampleBlock();
MutableColumns new_columns = block.cloneEmptyColumns();
while (StoragesInfo info = stream.next())
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription{{
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"name", std::make_shared<DataTypeString>()},
{"disk", std::make_shared<DataTypeString>()},
{"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
{"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
{"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt32>())}
}});
setInMemoryMetadata(storage_metadata);
}
Pipe read(
const Names & /* column_names */,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/) override
{
StoragesInfoStream stream(query_info, context);
/// Create the result.
Block block = metadata_snapshot->getSampleBlock();
MutableColumns new_columns = block.cloneEmptyColumns();
while (StoragesInfo info = stream.next())
const auto parts = info.data->getDetachedParts();
for (const auto & p : parts)
{
const auto parts = info.data->getDetachedParts();
for (const auto & p : parts)
{
size_t i = 0;
new_columns[i++]->insert(info.database);
new_columns[i++]->insert(info.table);
new_columns[i++]->insert(p.valid_name ? p.partition_id : Field());
new_columns[i++]->insert(p.dir_name);
new_columns[i++]->insert(p.disk);
new_columns[i++]->insert(p.valid_name ? p.prefix : Field());
new_columns[i++]->insert(p.valid_name ? p.min_block : Field());
new_columns[i++]->insert(p.valid_name ? p.max_block : Field());
new_columns[i++]->insert(p.valid_name ? p.level : Field());
}
size_t i = 0;
new_columns[i++]->insert(info.database);
new_columns[i++]->insert(info.table);
new_columns[i++]->insert(p.valid_name ? p.partition_id : Field());
new_columns[i++]->insert(p.dir_name);
new_columns[i++]->insert(p.disk);
new_columns[i++]->insert(p.valid_name ? p.prefix : Field());
new_columns[i++]->insert(p.valid_name ? p.min_block : Field());
new_columns[i++]->insert(p.valid_name ? p.max_block : Field());
new_columns[i++]->insert(p.valid_name ? p.level : Field());
}
UInt64 num_rows = new_columns.at(0)->size();
Chunk chunk(std::move(new_columns), num_rows);
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk)));
}
};
StoragePtr
createDetachedPartsTable()
{
return StorageSystemDetachedParts::create();
UInt64 num_rows = new_columns.at(0)->size();
Chunk chunk(std::move(new_columns), num_rows);
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk)));
}
}

View File

@ -1,10 +1,35 @@
#pragma once
#include <Storages/IStorage_fwd.h>
#include <Storages/IStorage.h>
namespace DB
{
StoragePtr createDetachedPartsTable();
/**
* Implements system table 'detached_parts' which allows to get information
* about detached data parts for tables of MergeTree family.
* We don't use StorageSystemPartsBase, because it introduces virtual _state
* column and column aliases which we don't need.
*/
class StorageSystemDetachedParts final :
public ext::shared_ptr_helper<StorageSystemDetachedParts>,
public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemDetachedParts>;
public:
std::string getName() const override { return "SystemDetachedParts"; }
protected:
explicit StorageSystemDetachedParts(const StorageID & table_id_);
Pipe read(
const Names & /* column_names */,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/) override;
};
}

View File

@ -11,8 +11,8 @@ namespace ErrorCodes
}
StorageSystemDisks::StorageSystemDisks(const std::string & name_)
: IStorage({"system", name_})
StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(

View File

@ -30,7 +30,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemDisks(const std::string & name_);
StorageSystemDisks(const StorageID & table_id_);
};
}

View File

@ -11,8 +11,8 @@ namespace DB
{
StorageSystemOne::StorageSystemOne(const std::string & name_)
: IStorage({"system", name_})
StorageSystemOne::StorageSystemOne(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription({{"dummy", std::make_shared<DataTypeUInt8>()}}));

View File

@ -31,7 +31,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemOne(const std::string & name_);
StorageSystemOne(const StorageID & table_id_);
};
}

View File

@ -15,8 +15,8 @@
namespace DB
{
StorageSystemParts::StorageSystemParts(const std::string & name_)
: StorageSystemPartsBase(name_,
StorageSystemParts::StorageSystemParts(const StorageID & table_id_)
: StorageSystemPartsBase(table_id_,
{
{"partition", std::make_shared<DataTypeString>()},
{"name", std::make_shared<DataTypeString>()},

View File

@ -19,7 +19,7 @@ public:
std::string getName() const override { return "SystemParts"; }
protected:
explicit StorageSystemParts(const std::string & name_);
explicit StorageSystemParts(const StorageID & table_id_);
void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override;
};

View File

@ -258,8 +258,8 @@ Pipe StorageSystemPartsBase::read(
}
StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_)
: IStorage(StorageID{"system", name_})
StorageSystemPartsBase::StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_)
: IStorage(table_id_)
{
ColumnsDescription tmp_columns(std::move(columns_));

View File

@ -72,7 +72,7 @@ private:
protected:
const FormatSettings format_settings;
StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_);
StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_);
virtual void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) = 0;
};

View File

@ -15,8 +15,8 @@ namespace DB
{
StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_)
: StorageSystemPartsBase(name_,
StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_)
: StorageSystemPartsBase(table_id_,
{
{"partition", std::make_shared<DataTypeString>()},
{"name", std::make_shared<DataTypeString>()},

View File

@ -21,7 +21,7 @@ public:
std::string getName() const override { return "SystemPartsColumns"; }
protected:
StorageSystemPartsColumns(const std::string & name_);
StorageSystemPartsColumns(const StorageID & table_id_);
void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override;
};

View File

@ -16,8 +16,8 @@ namespace DB
{
StorageSystemReplicas::StorageSystemReplicas(const std::string & name_)
: IStorage({"system", name_})
StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription({

View File

@ -28,7 +28,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemReplicas(const std::string & name_);
StorageSystemReplicas(const StorageID & table_id_);
};
}

View File

@ -126,8 +126,8 @@ namespace
}
StorageSystemStackTrace::StorageSystemStackTrace(const String & name_)
: IStorageSystemOneBlock<StorageSystemStackTrace>(name_)
StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
: IStorageSystemOneBlock<StorageSystemStackTrace>(table_id_)
{
notification_pipe.open();

View File

@ -23,7 +23,7 @@ public:
String getName() const override { return "SystemStackTrace"; }
static NamesAndTypesList getNamesAndTypes();
StorageSystemStackTrace(const String & name_);
StorageSystemStackTrace(const StorageID & table_id_);
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -17,8 +17,8 @@ namespace ErrorCodes
}
StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_)
: IStorage({"system", name_})
StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(

View File

@ -30,7 +30,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemStoragePolicies(const std::string & name_);
StorageSystemStoragePolicies(const StorageID & table_id_);
};
}

View File

@ -30,8 +30,8 @@ namespace ErrorCodes
}
StorageSystemTables::StorageSystemTables(const std::string & name_)
: IStorage({"system", name_})
StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(

View File

@ -28,7 +28,7 @@ public:
unsigned num_streams) override;
protected:
StorageSystemTables(const std::string & name_);
StorageSystemTables(const StorageID & table_id_);
};
}

View File

@ -1,5 +1,6 @@
#include <Databases/IDatabase.h>
#include <Storages/System/attachSystemTables.h>
#include <Storages/System/attachSystemTablesImpl.h>
#include <Storages/System/StorageSystemAggregateFunctionCombinators.h>
#include <Storages/System/StorageSystemAsynchronousMetrics.h>
@ -66,79 +67,80 @@
namespace DB
{
///TODO allow store system tables in DatabaseAtomic
void attachSystemTablesLocal(IDatabase & system_database)
{
system_database.attachTable("one", StorageSystemOne::create("one"));
system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false));
system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true));
system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false));
system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true));
system_database.attachTable("databases", StorageSystemDatabases::create("databases"));
system_database.attachTable("tables", StorageSystemTables::create("tables"));
system_database.attachTable("columns", StorageSystemColumns::create("columns"));
system_database.attachTable("functions", StorageSystemFunctions::create("functions"));
system_database.attachTable("events", StorageSystemEvents::create("events"));
system_database.attachTable("settings", StorageSystemSettings::create("settings"));
system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings"));
system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
system_database.attachTable("formats", StorageSystemFormats::create("formats"));
system_database.attachTable("table_functions", StorageSystemTableFunctions::create("table_functions"));
system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators"));
system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families"));
system_database.attachTable("collations", StorageSystemCollations::create("collations"));
system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines"));
system_database.attachTable("contributors", StorageSystemContributors::create("contributors"));
system_database.attachTable("users", StorageSystemUsers::create("users"));
system_database.attachTable("roles", StorageSystemRoles::create("roles"));
system_database.attachTable("grants", StorageSystemGrants::create("grants"));
system_database.attachTable("role_grants", StorageSystemRoleGrants::create("role_grants"));
system_database.attachTable("current_roles", StorageSystemCurrentRoles::create("current_roles"));
system_database.attachTable("enabled_roles", StorageSystemEnabledRoles::create("enabled_roles"));
system_database.attachTable("settings_profiles", StorageSystemSettingsProfiles::create("settings_profiles"));
system_database.attachTable("settings_profile_elements", StorageSystemSettingsProfileElements::create("settings_profile_elements"));
system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies"));
system_database.attachTable("quotas", StorageSystemQuotas::create("quotas"));
system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits"));
system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage"));
system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage"));
system_database.attachTable("privileges", StorageSystemPrivileges::create("privileges"));
attach<StorageSystemOne>(system_database, "one");
attach<StorageSystemNumbers>(system_database, "numbers", false);
attach<StorageSystemNumbers>(system_database, "numbers_mt", true);
attach<StorageSystemZeros>(system_database, "zeros", false);
attach<StorageSystemZeros>(system_database, "zeros_mt", true);
attach<StorageSystemDatabases>(system_database, "databases");
attach<StorageSystemTables>(system_database, "tables");
attach<StorageSystemColumns>(system_database, "columns");
attach<StorageSystemFunctions>(system_database, "functions");
attach<StorageSystemEvents>(system_database, "events");
attach<StorageSystemSettings>(system_database, "settings");
attach<SystemMergeTreeSettings>(system_database, "merge_tree_settings");
attach<StorageSystemBuildOptions>(system_database, "build_options");
attach<StorageSystemFormats>(system_database, "formats");
attach<StorageSystemTableFunctions>(system_database, "table_functions");
attach<StorageSystemAggregateFunctionCombinators>(system_database, "aggregate_function_combinators");
attach<StorageSystemDataTypeFamilies>(system_database, "data_type_families");
attach<StorageSystemCollations>(system_database, "collations");
attach<StorageSystemTableEngines>(system_database, "table_engines");
attach<StorageSystemContributors>(system_database, "contributors");
attach<StorageSystemUsers>(system_database, "users");
attach<StorageSystemRoles>(system_database, "roles");
attach<StorageSystemGrants>(system_database, "grants");
attach<StorageSystemRoleGrants>(system_database, "role_grants");
attach<StorageSystemCurrentRoles>(system_database, "current_roles");
attach<StorageSystemEnabledRoles>(system_database, "enabled_roles");
attach<StorageSystemSettingsProfiles>(system_database, "settings_profiles");
attach<StorageSystemSettingsProfileElements>(system_database, "settings_profile_elements");
attach<StorageSystemRowPolicies>(system_database, "row_policies");
attach<StorageSystemQuotas>(system_database, "quotas");
attach<StorageSystemQuotaLimits>(system_database, "quota_limits");
attach<StorageSystemQuotaUsage>(system_database, "quota_usage");
attach<StorageSystemQuotasUsage>(system_database, "quotas_usage");
attach<StorageSystemPrivileges>(system_database, "privileges");
#if !defined(ARCADIA_BUILD)
system_database.attachTable("licenses", StorageSystemLicenses::create("licenses"));
attach<StorageSystemLicenses>(system_database, "licenses");
#endif
#ifdef OS_LINUX
system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace"));
attach<StorageSystemStackTrace>(system_database, "stack_trace");
#endif
}
void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
{
attachSystemTablesLocal(system_database);
system_database.attachTable("parts", StorageSystemParts::create("parts"));
system_database.attachTable("detached_parts", createDetachedPartsTable());
system_database.attachTable("parts_columns", StorageSystemPartsColumns::create("parts_columns"));
system_database.attachTable("disks", StorageSystemDisks::create("disks"));
system_database.attachTable("storage_policies", StorageSystemStoragePolicies::create("storage_policies"));
system_database.attachTable("processes", StorageSystemProcesses::create("processes"));
system_database.attachTable("metrics", StorageSystemMetrics::create("metrics"));
system_database.attachTable("merges", StorageSystemMerges::create("merges"));
system_database.attachTable("mutations", StorageSystemMutations::create("mutations"));
system_database.attachTable("replicas", StorageSystemReplicas::create("replicas"));
system_database.attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
system_database.attachTable("distribution_queue", StorageSystemDistributionQueue::create("distribution_queue"));
system_database.attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries"));
system_database.attachTable("models", StorageSystemModels::create("models"));
system_database.attachTable("clusters", StorageSystemClusters::create("clusters"));
system_database.attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions"));
system_database.attachTable("macros", StorageSystemMacros::create("macros"));
attach<StorageSystemParts>(system_database, "parts");
attach<StorageSystemDetachedParts>(system_database, "detached_parts");
attach<StorageSystemPartsColumns>(system_database, "parts_columns");
attach<StorageSystemDisks>(system_database, "disks");
attach<StorageSystemStoragePolicies>(system_database, "storage_policies");
attach<StorageSystemProcesses>(system_database, "processes");
attach<StorageSystemMetrics>(system_database, "metrics");
attach<StorageSystemMerges>(system_database, "merges");
attach<StorageSystemMutations>(system_database, "mutations");
attach<StorageSystemReplicas>(system_database, "replicas");
attach<StorageSystemReplicationQueue>(system_database, "replication_queue");
attach<StorageSystemDistributionQueue>(system_database, "distribution_queue");
attach<StorageSystemDictionaries>(system_database, "dictionaries");
attach<StorageSystemModels>(system_database, "models");
attach<StorageSystemClusters>(system_database, "clusters");
attach<StorageSystemGraphite>(system_database, "graphite_retentions");
attach<StorageSystemMacros>(system_database, "macros");
if (has_zookeeper)
system_database.attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper"));
attach<StorageSystemZooKeeper>(system_database, "zookeeper");
}
void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics)
{
system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics));
attach<StorageSystemAsynchronousMetrics>(system_database, "asynchronous_metrics", async_metrics);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Databases/IDatabase.h>
namespace DB
{
template<typename StorageT, typename... StorageArgs>
void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args)
{
if (system_database.getUUID() == UUIDHelpers::Nil)
{
/// Attach to Ordinary database
auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name);
system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...));
}
else
{
/// Attach to Atomic database
/// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages
/// and path is actually not used
auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4());
String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid);
system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...), path);
}
}
}

View File

@ -26,7 +26,6 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va
/// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query.
/// Only elements of the outer conjunction are considered, depending only on the columns present in the block.
/// Returns true if at least one row is discarded.
void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context);
/// Extract from the input stream a set of `name` column values

View File

@ -49,6 +49,10 @@ def remove_control_characters(s):
s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s)
return s
def get_db_engine(args):
if args.atomic_db_engine:
return " ENGINE=Atomic"
return ""
def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file):
@ -69,7 +73,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
database = 'test_{suffix}'.format(suffix=random_str())
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE " + database)
clickhouse_proc_create.communicate("CREATE DATABASE " + database + get_db_engine(args))
os.environ["CLICKHOUSE_DATABASE"] = database
@ -507,10 +511,10 @@ def main(args):
if args.database and args.database != "test":
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args))
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test")
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args))
def is_test_from_dir(suite_dir, case):
case_file = os.path.join(suite_dir, case)
@ -722,6 +726,7 @@ if __name__ == '__main__':
parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started')
parser.add_argument('--skip-list-path', help="Path to skip-list file")
parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found")
parser.add_argument('--atomic-db-engine', action='store_true', help='Create databases with Atomic engine by default')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests')

View File

@ -2,7 +2,6 @@
<profiles>
<default>
<default_database_engine>Atomic</default_database_engine>
<allow_experimental_database_atomic>1</allow_experimental_database_atomic>
<show_table_uuid_in_table_create_query_if_not_nil>0</show_table_uuid_in_table_create_query_if_not_nil>
</default>
</profiles>

View File

@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > "$
# if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy
for SQL_FUZZY_RUN in $(seq "${SQL_FUZZY_RUNS:=10}"); do
env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" "$CURDIR"/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1
env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" "$CURDIR"/00746_sql_fuzzy.pl | timeout 60 $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1
if [[ $($CLICKHOUSE_CLIENT -q "SELECT 'Still alive'") != 'Still alive' ]]; then
break
fi

View File

@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107"
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01107 ENGINE=Atomic"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" &
@ -18,7 +18,7 @@ wait
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt"
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt"
$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107"
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107"
$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107"
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" &

View File

@ -1,5 +1,4 @@
DROP DATABASE IF EXISTS test_01109;
SET allow_experimental_database_atomic=1;
CREATE DATABASE test_01109 ENGINE=Atomic;
USE test_01109;

View File

@ -8,8 +8,8 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3"
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_1 ENGINE=Atomic"
$CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic"
$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01114_2"
$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1"

View File

@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_renamed"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_atomic"
$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" 2>&1| grep -F "does not support" > /dev/null && echo "ok"
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'"
$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'"
# 2. check metadata
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE DATABASE test_01192"
@ -40,7 +40,7 @@ $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW test_01192.mv TO test_01192.rmt
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number FROM numbers(10)" && echo "inserted"
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic"
$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic"
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01192_renamed"
# it's blocking
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01192.mt TO test_01192_atomic.mt, test_01192.rmt TO test_01192_atomic.rmt, test_01192.mv TO test_01192_atomic.mv" && echo "renamed"

View File

@ -0,0 +1,30 @@
DROP TABLE IF EXISTS test1;
DROP TABLE IF EXISTS test1_distributed;
DROP TABLE IF EXISTS test_merge;
SET enable_optimize_predicate_expression = 1;
CREATE TABLE test1 (id Int64, name String) ENGINE MergeTree PARTITION BY (id) ORDER BY (id);
CREATE TABLE test1_distributed AS test1 ENGINE = Distributed(test_cluster_two_shards_localhost, default, test1);
CREATE TABLE test_merge AS test1 ENGINE = Merge('default', 'test1_distributed');
SELECT count() FROM test_merge
JOIN (SELECT 'anystring' AS name) AS n
USING name
WHERE id = 1;
DROP TABLE test1;
DROP TABLE test_merge;
CREATE TABLE test1 (id Int64, name String) ENGINE MergeTree PARTITION BY (id) ORDER BY (id);
CREATE TABLE test_merge AS test1 ENGINE = Merge('default', 'test1');
SELECT count() FROM test_merge
JOIN (SELECT 'anystring' AS name) AS n
USING name
WHERE id = 1;
DROP TABLE test1;
DROP TABLE test_merge;
DROP TABLE test1_distributed;

View File

@ -0,0 +1,10 @@
0 0
1 0
2 0
3 0
4 0
5 0
6 0
7 0
8 0
9 0

View File

@ -0,0 +1 @@
SELECT number % 100 AS k, sumArray(emptyArrayUInt8()) AS v FROM numbers(10) GROUP BY k;

View File

@ -68,3 +68,9 @@ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | gre
# There shouldn't be any code snippets under GPL or LGPL
find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL"
# There shouldn't be any docker containers outside docker directory
find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:"
# There shouldn't be any docker compose files outside docker directory
#find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:"