Merge branch 'master' into roaring-memory-tracker

This commit is contained in:
mergify[bot] 2021-08-08 15:19:57 +00:00 committed by GitHub
commit 8d05b7c5e5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
163 changed files with 706 additions and 301 deletions

3
.gitmodules vendored
View File

@ -243,3 +243,6 @@
[submodule "contrib/s2geometry"]
path = contrib/s2geometry
url = https://github.com/ClickHouse-Extras/s2geometry.git
[submodule "contrib/bzip2"]
path = contrib/bzip2
url = https://github.com/ClickHouse-Extras/bzip2.git

View File

@ -543,6 +543,7 @@ include (cmake/find/nuraft.cmake)
include (cmake/find/yaml-cpp.cmake)
include (cmake/find/s2geometry.cmake)
include (cmake/find/nlp.cmake)
include (cmake/find/bzip2.cmake)
if(NOT USE_INTERNAL_PARQUET_LIBRARY)
set (ENABLE_ORC OFF CACHE INTERNAL "")

19
cmake/find/bzip2.cmake Normal file
View File

@ -0,0 +1,19 @@
option(ENABLE_BZIP2 "Enable bzip2 compression support" ${ENABLE_LIBRARIES})
if (NOT ENABLE_BZIP2)
message (STATUS "bzip2 compression disabled")
return()
endif()
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/bzip2/bzlib.h")
message (WARNING "submodule contrib/bzip2 is missing. to fix try run: \n git submodule update --init --recursive")
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal bzip2 library")
set (USE_NLP 0)
return()
endif ()
set (USE_BZIP2 1)
set (BZIP2_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2")
set (BZIP2_LIBRARY bzip2)
message (STATUS "Using bzip2=${USE_BZIP2}: ${BZIP2_INCLUDE_DIR} : ${BZIP2_LIBRARY}")

View File

@ -334,6 +334,10 @@ if (USE_NLP)
add_subdirectory(lemmagen-c-cmake)
endif()
if (USE_BZIP2)
add_subdirectory(bzip2-cmake)
endif()
if (USE_SQLITE)
add_subdirectory(sqlite-cmake)
endif()

1
contrib/bzip2 vendored Submodule

@ -0,0 +1 @@
Subproject commit bf905ea2251191ff9911ae7ec0cfc35d41f9f7f6

View File

@ -0,0 +1,23 @@
set(BZIP2_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2")
set(BZIP2_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/bzip2")
set(SRCS
"${BZIP2_SOURCE_DIR}/blocksort.c"
"${BZIP2_SOURCE_DIR}/huffman.c"
"${BZIP2_SOURCE_DIR}/crctable.c"
"${BZIP2_SOURCE_DIR}/randtable.c"
"${BZIP2_SOURCE_DIR}/compress.c"
"${BZIP2_SOURCE_DIR}/decompress.c"
"${BZIP2_SOURCE_DIR}/bzlib.c"
)
# From bzip2/CMakeLists.txt
set(BZ_VERSION "1.0.7")
configure_file (
"${BZIP2_SOURCE_DIR}/bz_version.h.in"
"${BZIP2_BINARY_DIR}/bz_version.h"
)
add_library(bzip2 ${SRCS})
target_include_directories(bzip2 PUBLIC "${BZIP2_SOURCE_DIR}" "${BZIP2_BINARY_DIR}")

View File

@ -312,6 +312,7 @@ function run_tests
01798_uniq_theta_sketch
01799_long_uniq_theta_sketch
01890_stem # depends on libstemmer_c
02003_compress_bz2 # depends on bzip2
collate
collation
_orc_

View File

@ -481,6 +481,11 @@ if (USE_NLP)
dbms_target_link_libraries (PUBLIC lemmagen)
endif()
if (USE_BZIP2)
target_link_libraries (clickhouse_common_io PRIVATE ${BZIP2_LIBRARY})
target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BZIP2_INCLUDE_DIR})
endif()
include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake")
if (ENABLE_TESTS AND USE_GTEST)

View File

@ -561,6 +561,8 @@
M(591, SQLITE_ENGINE_ERROR) \
M(592, DATA_ENCRYPTION_ERROR) \
M(593, ZERO_COPY_REPLICATION_ERROR) \
M(594, BZIP2_STREAM_DECODER_FAILED) \
M(595, BZIP2_STREAM_ENCODER_FAILED) \
\
M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \

View File

@ -19,3 +19,4 @@
#cmakedefine01 USE_DATASKETCHES
#cmakedefine01 USE_YAML_CPP
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY
#cmakedefine01 USE_BZIP2

View File

@ -169,6 +169,7 @@ class IColumn;
M(Int64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \
\
M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \
M(Bool, log_formatted_queries, 0, "Log formatted queries and write the log to the system table.", 0) \
M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \
M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log.", 0) \
M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \

View File

@ -0,0 +1,97 @@
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
#if USE_BZIP2
# include <IO/Bzip2ReadBuffer.h>
# include <bzlib.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BZIP2_STREAM_DECODER_FAILED;
}
class Bzip2ReadBuffer::Bzip2StateWrapper
{
public:
Bzip2StateWrapper()
{
memset(&stream, 0, sizeof(stream));
int ret = BZ2_bzDecompressInit(&stream, 0, 0);
if (ret != BZ_OK)
throw Exception(
ErrorCodes::BZIP2_STREAM_DECODER_FAILED,
"bzip2 stream encoder init failed: error code: {}",
ret);
}
~Bzip2StateWrapper()
{
BZ2_bzDecompressEnd(&stream);
}
bz_stream stream;
};
Bzip2ReadBuffer::Bzip2ReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t buf_size, char *existing_memory, size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(std::move(in_))
, bz(std::make_unique<Bzip2StateWrapper>())
, eof(false)
{
}
Bzip2ReadBuffer::~Bzip2ReadBuffer() = default;
bool Bzip2ReadBuffer::nextImpl()
{
if (eof)
return false;
if (!bz->stream.avail_in)
{
in->nextIfAtEnd();
bz->stream.avail_in = in->buffer().end() - in->position();
bz->stream.next_in = in->position();
}
bz->stream.avail_out = internal_buffer.size();
bz->stream.next_out = internal_buffer.begin();
int ret = BZ2_bzDecompress(&bz->stream);
in->position() = in->buffer().end() - bz->stream.avail_in;
working_buffer.resize(internal_buffer.size() - bz->stream.avail_out);
if (ret == BZ_STREAM_END)
{
if (in->eof())
{
eof = true;
return !working_buffer.empty();
}
else
{
throw Exception(
ErrorCodes::BZIP2_STREAM_DECODER_FAILED,
"bzip2 decoder finished, but input stream has not exceeded: error code: {}", ret);
}
}
if (ret != BZ_OK)
throw Exception(
ErrorCodes::BZIP2_STREAM_DECODER_FAILED,
"bzip2 stream decoder failed: error code: {}",
ret);
return true;
}
}
#endif

33
src/IO/Bzip2ReadBuffer.h Normal file
View File

@ -0,0 +1,33 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
class Bzip2ReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{
public:
Bzip2ReadBuffer(
std::unique_ptr<ReadBuffer> in_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
~Bzip2ReadBuffer() override;
private:
bool nextImpl() override;
std::unique_ptr<ReadBuffer> in;
class Bzip2StateWrapper;
std::unique_ptr<Bzip2StateWrapper> bz;
bool eof;
};
}

138
src/IO/Bzip2WriteBuffer.cpp Normal file
View File

@ -0,0 +1,138 @@
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
#if USE_BROTLI
# include <IO/Bzip2WriteBuffer.h>
# include <bzlib.h>
#include <Common/MemoryTracker.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BZIP2_STREAM_ENCODER_FAILED;
}
class Bzip2WriteBuffer::Bzip2StateWrapper
{
public:
explicit Bzip2StateWrapper(int compression_level)
{
memset(&stream, 0, sizeof(stream));
int ret = BZ2_bzCompressInit(&stream, compression_level, 0, 0);
if (ret != BZ_OK)
throw Exception(
ErrorCodes::BZIP2_STREAM_ENCODER_FAILED,
"bzip2 stream encoder init failed: error code: {}",
ret);
}
~Bzip2StateWrapper()
{
BZ2_bzCompressEnd(&stream);
}
bz_stream stream;
};
Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr<WriteBuffer> out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, bz(std::make_unique<Bzip2StateWrapper>(compression_level))
, out(std::move(out_))
{
}
Bzip2WriteBuffer::~Bzip2WriteBuffer()
{
/// FIXME move final flush into the caller
MemoryTracker::LockExceptionInThread lock(VariableContext::Global);
finish();
}
void Bzip2WriteBuffer::nextImpl()
{
if (!offset())
{
return;
}
bz->stream.next_in = working_buffer.begin();
bz->stream.avail_in = offset();
try
{
do
{
out->nextIfAtEnd();
bz->stream.next_out = out->position();
bz->stream.avail_out = out->buffer().end() - out->position();
int ret = BZ2_bzCompress(&bz->stream, BZ_RUN);
out->position() = out->buffer().end() - bz->stream.avail_out;
if (ret != BZ_RUN_OK)
throw Exception(
ErrorCodes::BZIP2_STREAM_ENCODER_FAILED,
"bzip2 stream encoder failed: error code: {}",
ret);
}
while (bz->stream.avail_in > 0);
}
catch (...)
{
/// Do not try to write next time after exception.
out->position() = out->buffer().begin();
throw;
}
}
void Bzip2WriteBuffer::finish()
{
if (finished)
return;
try
{
finishImpl();
out->finalize();
finished = true;
}
catch (...)
{
/// Do not try to flush next time after exception.
out->position() = out->buffer().begin();
finished = true;
throw;
}
}
void Bzip2WriteBuffer::finishImpl()
{
next();
out->nextIfAtEnd();
bz->stream.next_out = out->position();
bz->stream.avail_out = out->buffer().end() - out->position();
int ret = BZ2_bzCompress(&bz->stream, BZ_FINISH);
out->position() = out->buffer().end() - bz->stream.avail_out;
if (ret != BZ_STREAM_END && ret != BZ_FINISH_OK)
throw Exception(
ErrorCodes::BZIP2_STREAM_ENCODER_FAILED,
"bzip2 stream encoder failed: error code: {}",
ret);
}
}
#endif

37
src/IO/Bzip2WriteBuffer.h Normal file
View File

@ -0,0 +1,37 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
class Bzip2WriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
public:
Bzip2WriteBuffer(
std::unique_ptr<WriteBuffer> out_,
int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
~Bzip2WriteBuffer() override;
void finalize() override { finish(); }
private:
void nextImpl() override;
void finish();
void finishImpl();
class Bzip2StateWrapper;
std::unique_ptr<Bzip2StateWrapper> bz;
std::unique_ptr<WriteBuffer> out;
bool finished = false;
};
}

View File

@ -10,6 +10,8 @@
#include <IO/ZlibInflatingReadBuffer.h>
#include <IO/ZstdDeflatingWriteBuffer.h>
#include <IO/ZstdInflatingReadBuffer.h>
#include <IO/Bzip2ReadBuffer.h>
#include <IO/Bzip2WriteBuffer.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
@ -40,6 +42,8 @@ std::string toContentEncodingName(CompressionMethod method)
return "xz";
case CompressionMethod::Zstd:
return "zstd";
case CompressionMethod::Bzip2:
return "bz2";
case CompressionMethod::None:
return "";
}
@ -69,11 +73,13 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s
return CompressionMethod::Xz;
if (method_str == "zstd" || method_str == "zst")
return CompressionMethod::Zstd;
if (method_str == "bz2")
return CompressionMethod::Bzip2;
if (hint.empty() || hint == "auto" || hint == "none")
return CompressionMethod::None;
throw Exception(
"Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd' are supported as compression methods",
"Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'bz2' are supported as compression methods",
ErrorCodes::NOT_IMPLEMENTED);
}
@ -91,7 +97,10 @@ std::unique_ptr<ReadBuffer> wrapReadBufferWithCompressionMethod(
return std::make_unique<LZMAInflatingReadBuffer>(std::move(nested), buf_size, existing_memory, alignment);
if (method == CompressionMethod::Zstd)
return std::make_unique<ZstdInflatingReadBuffer>(std::move(nested), buf_size, existing_memory, alignment);
#if USE_BZIP2
if (method == CompressionMethod::Bzip2)
return std::make_unique<Bzip2ReadBuffer>(std::move(nested), buf_size, existing_memory, alignment);
#endif
if (method == CompressionMethod::None)
return nested;
@ -114,7 +123,10 @@ std::unique_ptr<WriteBuffer> wrapWriteBufferWithCompressionMethod(
if (method == CompressionMethod::Zstd)
return std::make_unique<ZstdDeflatingWriteBuffer>(std::move(nested), level, buf_size, existing_memory, alignment);
#if USE_BZIP2
if (method == CompressionMethod::Bzip2)
return std::make_unique<Bzip2WriteBuffer>(std::move(nested), level, buf_size, existing_memory, alignment);
#endif
if (method == CompressionMethod::None)
return nested;

View File

@ -31,7 +31,8 @@ enum class CompressionMethod
/// Zstd compressor
/// This option corresponds to HTTP Content-Encoding: zstd
Zstd,
Brotli
Brotli,
Bzip2
};
/// How the compression method is named in HTTP.

View File

@ -23,6 +23,8 @@ SRCS(
AIOContextPool.cpp
BrotliReadBuffer.cpp
BrotliWriteBuffer.cpp
Bzip2ReadBuffer.cpp
Bzip2WriteBuffer.cpp
CascadeWriteBuffer.cpp
CompressionMethod.cpp
DoubleConverter.cpp

View File

@ -1378,7 +1378,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
/// Limit is no longer needed if there is prelimit.
///
/// NOTE: that LIMIT cannot be applied of OFFSET should not be applied,
/// NOTE: that LIMIT cannot be applied if OFFSET should not be applied,
/// since LIMIT will apply OFFSET too.
/// This is the case for various optimizations for distributed queries,
/// and when LIMIT cannot be applied it will be applied on the initiator anyway.

View File

@ -57,6 +57,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes()
{"current_database", std::make_shared<DataTypeString>()},
{"query", std::make_shared<DataTypeString>()},
{"formatted_query", std::make_shared<DataTypeString>()},
{"normalized_query_hash", std::make_shared<DataTypeUInt64>()},
{"query_kind", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"databases", std::make_shared<DataTypeArray>(
@ -151,6 +152,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insertData(current_database.data(), current_database.size());
columns[i++]->insertData(query.data(), query.size());
columns[i++]->insertData(formatted_query.data(), formatted_query.size());
columns[i++]->insert(normalized_query_hash);
columns[i++]->insertData(query_kind.data(), query_kind.size());

View File

@ -51,6 +51,7 @@ struct QueryLogElement
String current_database;
String query;
String formatted_query;
UInt64 normalized_query_hash{};
String query_kind;

View File

@ -265,7 +265,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr
// Try log query_kind if ast is valid
if (ast)
{
elem.query_kind = ast->getQueryKindString();
if (settings.log_formatted_queries)
elem.formatted_query = queryToString(ast);
}
// We don't calculate databases, tables and columns when the query isn't able to start
@ -641,6 +645,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.current_database = context->getCurrentDatabase();
elem.query = query_for_logging;
if (settings.log_formatted_queries)
elem.formatted_query = queryToString(ast);
elem.normalized_query_hash = normalizedQueryHash<false>(query_for_logging);
elem.client_info = client_info;

View File

@ -195,8 +195,8 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc
, log(&Poco::Logger::get("NuKeeperTCPHandler"))
, global_context(Context::createCopy(server.context()))
, keeper_dispatcher(global_context->getKeeperStorageDispatcher())
, operation_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000)
, session_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000)
, operation_timeout(0, global_context->getConfigRef().getUInt("keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000)
, session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000)
, poll_wrapper(std::make_unique<SocketInterruptablePollWrapper>(socket_))
, responses(std::make_unique<ThreadSafeResponseQueue>())
{

View File

@ -124,7 +124,7 @@ struct Settings;
M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \
M(String, storage_policy, "default", "Name of storage disk policy", 0) \
M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
M(Bool, allow_remote_fs_zero_copy_replication, false, "Allow Zero-copy replication over remote fs", 0) \
M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \
M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \
M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \

View File

@ -144,9 +144,14 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState()
if (current_replica_index_tmp < 0 || active_replicas_tmp.size() < 2)
{
LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!");
/// we can reset the settings w/o lock (it's atomic)
execute_merges_on_single_replica_time_threshold = 0;
if (execute_merges_on_single_replica_time_threshold > 0)
{
LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use 'execute_merges_on_single_replica_time_threshold'");
/// we can reset the settings w/o lock (it's atomic)
execute_merges_on_single_replica_time_threshold = 0;
}
/// default value of remote_fs_execute_merges_on_single_replica_time_threshold is not 0
/// so we write no warning in log here
remote_fs_execute_merges_on_single_replica_time_threshold = 0;
return;
}

View File

@ -553,7 +553,6 @@ std::optional<QueryProcessingStage::Enum> StorageDistributed::getOptimizedQueryP
if (const ASTPtr order_by = select.orderBy())
return default_stage;
// LIMIT BY
// LIMIT
// OFFSET
if (select.limitLength() || select.limitOffset())

View File

@ -50,6 +50,7 @@ const char * auto_config_build[]
"USE_LDAP", "@USE_LDAP@",
"TZDATA_VERSION", "@TZDATA_VERSION@",
"USE_KRB5", "@USE_KRB5@",
"USE_BZIP2", "@USE_BZIP2@",
nullptr, nullptr
};

View File

@ -59,7 +59,6 @@
<merge_tree>
<min_bytes_for_wide_part>1024000</min_bytes_for_wide_part>
<old_parts_lifetime>1</old_parts_lifetime>
<allow_remote_fs_zero_copy_replication>1</allow_remote_fs_zero_copy_replication>
</merge_tree>
<remote_servers>

View File

@ -21,7 +21,6 @@
<merge_tree>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
<allow_remote_fs_zero_copy_replication>1</allow_remote_fs_zero_copy_replication>
</merge_tree>
<remote_servers>

View File

@ -66,7 +66,6 @@
<merge_tree>
<min_bytes_for_wide_part>1024</min_bytes_for_wide_part>
<old_parts_lifetime>1</old_parts_lifetime>
<allow_remote_fs_zero_copy_replication>1</allow_remote_fs_zero_copy_replication>
</merge_tree>
<remote_servers>

View File

@ -1,22 +1,22 @@
d Date
k UInt64
i32 Int32
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 10 42
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 9 41 1992-01-01 08:00:00
2015-01-01 10 42 1970-01-01 00:00:00
d Date
@ -25,14 +25,14 @@ i32 Int32
dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14']
2015-01-01 9 41 1992-01-01 08:00:00 [] []
2015-01-01 10 42 1970-01-01 00:00:00 [] []
@ -43,7 +43,7 @@ dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
@ -51,7 +51,7 @@ dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01']
2015-01-01 9 41 1992-01-01 08:00:00 [] [] []
@ -64,7 +64,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
@ -73,7 +73,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 0
@ -86,7 +86,7 @@ dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
s Int64 DEFAULT \'0\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
@ -94,7 +94,7 @@ dt DateTime(\'UTC\')
n.ui8 Array(UInt8)
n.s Array(String)
s Int64 DEFAULT \'0\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0
@ -108,7 +108,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
s UInt32 DEFAULT \'0\'
n.d Array(Date)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
@ -117,7 +117,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
s UInt32 DEFAULT \'0\'
n.d Array(Date)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['1970-01-01','1970-01-01','1970-01-01']
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['1970-01-01','1970-01-01','1970-01-01']
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['1970-01-01','1970-01-01','1970-01-01']
@ -129,14 +129,14 @@ i32 Int32
dt DateTime(\'UTC\')
n.s Array(String)
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
n.s Array(String)
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0
2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0
@ -147,13 +147,13 @@ k UInt64
i32 Int32
dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500
2015-01-01 7 39 2014-07-14 13:26:50 0
2015-01-01 8 40 2012-12-12 12:12:12 0
@ -166,7 +166,7 @@ dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
n.s Array(String)
n.d Array(Date)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
@ -174,7 +174,7 @@ dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
n.s Array(String)
n.d Array(Date)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500 [] []
2015-01-01 7 39 2014-07-14 13:26:50 0 [] []
2015-01-01 8 40 2012-12-12 12:12:12 0 [] []
@ -185,13 +185,13 @@ k UInt64
i32 Int32
dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime(\'UTC\')
s UInt32 DEFAULT \'0\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500
2015-01-01 7 39 2014-07-14 13:26:50 0
2015-01-01 8 40 2012-12-12 12:12:12 0
@ -202,13 +202,13 @@ k UInt64
i32 Int32
dt Date
s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192)
CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt Date
s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192)
CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 1970-01-02 03:55:00
2015-01-01 7 39 2014-07-14 1970-01-01 00:00:00
2015-01-01 8 40 2012-12-12 1970-01-01 00:00:00

View File

@ -3,8 +3,8 @@ DROP TABLE IF EXISTS replicated_alter2;
SET replication_alter_partitions_sync = 2;
CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r1', d, k, 8192);
CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r2', d, k, 8192);
CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r1', d, k, 8192);
CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r2', d, k, 8192);
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 10, 42);

View File

@ -56,7 +56,7 @@ CREATE TABLE aggregating_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE replicated_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE replicated_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y);
CREATE TABLE replicated_versioned_collapsing_merge_tree
@ -69,7 +69,7 @@ CREATE TABLE replicated_aggregating_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00083/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE replicated_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE replicated_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y);
CREATE TABLE replicated_versioned_collapsing_merge_tree_with_sampling

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS alter_00121;
CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t1', 'r1', d, (d), 8192);
CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192);
INSERT INTO alter_00121 VALUES ('2014-01-01', 1);
ALTER TABLE alter_00121 DROP COLUMN x;
DROP TABLE alter_00121;
CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t2', 'r1', d, (d), 8192);
CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192);
INSERT INTO alter_00121 VALUES ('2014-01-01');
SELECT * FROM alter_00121 ORDER BY d;

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS primary_key;
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00215/primary_key', 'r1', d, -x, 1);
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00215/primary_key', 'r1', d, -x, 1);
INSERT INTO primary_key (x) VALUES (1), (2), (3);
INSERT INTO primary_key (x) VALUES (1), (3), (2);

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS deduplication;
CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00226/deduplication', 'r1', d, x, 1);
CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00226/deduplication', 'r1', d, x, 1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);

View File

@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2;
DROP TABLE IF EXISTS attach_r1;
DROP TABLE IF EXISTS attach_r2;
CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r1', d, d, 8192);
CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r2', d, d, 8192);
CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r1', d, d, 8192);
CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r2', d, d, 8192);
INSERT INTO attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01');

View File

@ -24,8 +24,8 @@ SELECT '===Replicated case===';
DROP TABLE IF EXISTS clear_column1;
DROP TABLE IF EXISTS clear_column2;
SELECT sleep(1) FORMAT Null;
CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0;
CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0;
CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0;
CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01');
SYSTEM SYNC REPLICA clear_column2;

View File

@ -4,8 +4,8 @@ SELECT '*** Not partitioned ***';
DROP TABLE IF EXISTS not_partitioned_replica1_00502;
DROP TABLE IF EXISTS not_partitioned_replica2_00502;
CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '1') ORDER BY x;
CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '2') ORDER BY x;
CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x;
CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x;
INSERT INTO not_partitioned_replica1_00502 VALUES (1), (2), (3);
INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5);
@ -30,8 +30,8 @@ SELECT '*** Partitioned by week ***';
DROP TABLE IF EXISTS partitioned_by_week_replica1;
DROP TABLE IF EXISTS partitioned_by_week_replica2;
CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x;
CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x;
CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x;
CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x;
-- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02
INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3);
@ -57,8 +57,8 @@ SELECT '*** Partitioned by a (Date, UInt8) tuple ***';
DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502;
DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502;
CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x);
CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x);
CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x);
CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x);
INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3);
INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5);
@ -84,8 +84,8 @@ SELECT '*** Partitioned by String ***';
DROP TABLE IF EXISTS partitioned_by_string_replica1;
DROP TABLE IF EXISTS partitioned_by_string_replica2;
CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x;
CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x;
CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x;
CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x;
INSERT INTO partitioned_by_string_replica1 VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3);
INSERT INTO partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5);
@ -110,8 +110,8 @@ SELECT '*** Table without columns with fixed size ***';
DROP TABLE IF EXISTS without_fixed_size_columns_replica1;
DROP TABLE IF EXISTS without_fixed_size_columns_replica2;
CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s;
CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s;
CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s;
CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s;
INSERT INTO without_fixed_size_columns_replica1 VALUES ('a'), ('aa'), ('b'), ('cc');

View File

@ -5,7 +5,7 @@ SELECT '*** Replicated with sampling ***';
DROP TABLE IF EXISTS replicated_with_sampling;
CREATE TABLE replicated_with_sampling(x UInt8)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/replicated_with_sampling', 'r1')
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/replicated_with_sampling', 'r1')
ORDER BY x
SAMPLE BY x;
@ -72,7 +72,7 @@ SELECT '*** Table definition with SETTINGS ***';
DROP TABLE IF EXISTS with_settings;
CREATE TABLE with_settings(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/with_settings', 'r1')
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/with_settings', 'r1')
ORDER BY x
SETTINGS replicated_can_become_leader = 0;

View File

@ -4,9 +4,9 @@ DROP TABLE IF EXISTS with_deduplication_mv;
DROP TABLE IF EXISTS without_deduplication_mv;
CREATE TABLE with_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/with_deduplication', 'r1') ORDER BY x;
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x;
CREATE TABLE without_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0;
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0;
CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001'
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS deduplication_by_partition;
CREATE TABLE deduplication_by_partition(d Date, x UInt32) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/test_00516/deduplication_by_partition', 'r1', d, x, 8192);
ReplicatedMergeTree('/clickhouse/tables/{database}/test_00516/deduplication_by_partition', 'r1', d, x, 8192);
INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 1);
INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 2), ('2000-01-01', 3);

View File

@ -1,6 +1,6 @@
-- Check that settings are correctly passed through Distributed table
DROP TABLE IF EXISTS simple;
CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test_00563/tables/simple', '1') ORDER BY d;
CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00563/tables/simple', '1') ORDER BY d;
-- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix
INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1);

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS replicated_truncate1;
DROP TABLE IF EXISTS replicated_truncate2;
CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r1', d, k, 8192);
CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r2', d, k, 8192);
CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r1', d, k, 8192);
CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r2', d, k, 8192);
SELECT '======Before Truncate======';
INSERT INTO replicated_truncate1 VALUES ('2015-01-01', 10, 42);

View File

@ -3,7 +3,7 @@ CREATE TABLE default.cast1
`x` UInt8,
`e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1')
ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/test_00643/cast', 'r1')
ORDER BY e
SETTINGS index_granularity = 8192
x UInt8

View File

@ -22,7 +22,7 @@ CREATE TABLE cast1
'world' = 2
)
)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') ORDER BY e;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r1') ORDER BY e;
SHOW CREATE TABLE cast1 FORMAT TSVRaw;
DESC TABLE cast1;
@ -30,7 +30,7 @@ DESC TABLE cast1;
INSERT INTO cast1 (x) VALUES (1);
SELECT * FROM cast1;
CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r2') ORDER BY e;
CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r2') ORDER BY e;
SYSTEM SYNC REPLICA cast2;

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS byte_identical_r1;
DROP TABLE IF EXISTS byte_identical_r2;
CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r1') ORDER BY x;
CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r2') ORDER BY x;
CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r1') ORDER BY x;
CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r2') ORDER BY x;
INSERT INTO byte_identical_r1(x) VALUES (1), (2), (3);
SYSTEM SYNC REPLICA byte_identical_r2;

View File

@ -87,4 +87,4 @@ FORMAT PrettyCompactNoEscapes;
DROP TABLE IF EXISTS check_query_comment_column;
-- TODO: add here tests with ReplicatedMergeTree
-- TODO: add here tests with ReplicatedMergeTree({database})

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y;
INSERT INTO quorum1 VALUES (1, '1990-11-15');
INSERT INTO quorum1 VALUES (2, '1990-11-15');

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y;
SET insert_quorum=2, insert_quorum_parallel=0;
SET select_sequential_consistency=1;

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y;
SET insert_quorum=2, insert_quorum_parallel=0;
SET select_sequential_consistency=1;

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y;
INSERT INTO quorum1 VALUES (1, '1990-11-15');
INSERT INTO quorum1 VALUES (2, '1990-11-15');

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '2') ORDER BY x PARTITION BY y;
SET insert_quorum=2, insert_quorum_parallel=0;
SET select_sequential_consistency=1;

View File

@ -3,8 +3,8 @@ SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '2') ORDER BY x PARTITION BY y;
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '1') ORDER BY x PARTITION BY y;
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '2') ORDER BY x PARTITION BY y;
SET insert_quorum=2, insert_quorum_parallel=0;
SET select_sequential_consistency=1;

View File

@ -1,6 +1,6 @@
CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192
CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192
column_name1 UInt8 DEFAULT 1 comment
column_name2 UInt8 non default comment
CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192
CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192
column_name1 UInt8 DEFAULT 1 another comment
column_name2 UInt8 non default comment

View File

@ -4,7 +4,7 @@ CREATE TABLE check_comments
(
column_name1 UInt8 DEFAULT 1 COMMENT 'comment',
column_name2 UInt8 COMMENT 'non default comment'
) ENGINE = ReplicatedMergeTree('clickhouse/tables/test_00753/comments', 'r1')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00753/comments', 'r1')
ORDER BY column_name1;
SHOW CREATE check_comments;

View File

@ -9,6 +9,6 @@
1 2 1 30
1 2 4 90
*** Check SHOW CREATE TABLE ***
CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192
CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192
*** Check SHOW CREATE TABLE after offline ALTER ***
CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192
CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192

View File

@ -3,14 +3,14 @@ SET optimize_on_insert = 0;
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS old_style;
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00754/old_style', 'r1', d, x, 8192);
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00754/old_style', 'r1', d, x, 8192);
ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
DROP TABLE old_style;
DROP TABLE IF EXISTS summing_r1;
DROP TABLE IF EXISTS summing_r2;
CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r1') ORDER BY (x, y);
CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r2') ORDER BY (x, y);
CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r1') ORDER BY (x, y);
CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r2') ORDER BY (x, y);
/* Can't add an expression with existing column to ORDER BY. */
ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 }

View File

@ -1,5 +1,5 @@
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
1 2
1 2
1 2
@ -14,8 +14,8 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I
3 2
19 9
65 75
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
1 2
1 4
1 5
@ -28,10 +28,10 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I
3 2
19 9
65 75
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
1 2
1 4
1 5
@ -44,14 +44,14 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I
3 2
19 9
65 75
CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
1 2
1 3
1 2
1 3
CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192
CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192
1 2
1 3
1 2

View File

@ -9,14 +9,14 @@ CREATE TABLE minmax_idx
(
u64 UInt64,
i32 Int32
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r1')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r1')
ORDER BY u64;
CREATE TABLE minmax_idx_r
(
u64 UInt64,
i32 Int32
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r2')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r2')
ORDER BY u64;
INSERT INTO minmax_idx VALUES (1, 2);
@ -74,7 +74,7 @@ CREATE TABLE minmax_idx2
i32 Int32,
INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,
INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r1')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r1')
ORDER BY u64;
CREATE TABLE minmax_idx2_r
@ -83,7 +83,7 @@ CREATE TABLE minmax_idx2_r
i32 Int32,
INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,
INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r2')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r2')
ORDER BY u64;

View File

@ -15,7 +15,7 @@ CREATE TABLE minmax_idx1
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r1')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r1')
ORDER BY u64
SETTINGS index_granularity = 2;
@ -32,7 +32,7 @@ CREATE TABLE minmax_idx2
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r2')
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r2')
ORDER BY u64
SETTINGS index_granularity = 2;

View File

@ -20,7 +20,7 @@
274972506.6
9175437371954010821
9175437371954010821
CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
1.5555555555555 hello world! [77] ['John']
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
!

View File

@ -11,7 +11,7 @@ CREATE TABLE compression_codec_replicated1(
somenum Float64 CODEC(ZSTD(2)),
somestr FixedString(3) CODEC(LZ4HC(7)),
othernum Int64 CODEC(Delta)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '1') ORDER BY tuple();
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '1') ORDER BY tuple();
CREATE TABLE compression_codec_replicated2(
id UInt64 CODEC(LZ4),
@ -20,7 +20,7 @@ CREATE TABLE compression_codec_replicated2(
somenum Float64 CODEC(ZSTD(2)),
somestr FixedString(3) CODEC(LZ4HC(7)),
othernum Int64 CODEC(Delta)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '2') ORDER BY tuple();
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '2') ORDER BY tuple();
INSERT INTO compression_codec_replicated1 VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5);
@ -57,14 +57,14 @@ CREATE TABLE compression_codec_multiple_replicated1 (
data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)),
ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC),
somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '1') ORDER BY tuple();
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '1') ORDER BY tuple();
CREATE TABLE compression_codec_multiple_replicated2 (
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)),
data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)),
ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC),
somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '2') ORDER BY tuple();
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '2') ORDER BY tuple();
INSERT INTO compression_codec_multiple_replicated2 VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3);
@ -106,7 +106,7 @@ CREATE TABLE compression_codec_multiple_more_types_replicated (
id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC),
data FixedString(12) CODEC(ZSTD, ZSTD, Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC),
ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8))
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple();
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple();
SHOW CREATE TABLE compression_codec_multiple_more_types_replicated;
@ -124,7 +124,7 @@ CREATE TABLE compression_codec_multiple_with_key_replicated (
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta),
id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta),
data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!');

View File

@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2;
CREATE TABLE alter_compression_codec1 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id;
CREATE TABLE alter_compression_codec2 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id;
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1);
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2);

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS replicated_optimize1;
DROP TABLE IF EXISTS replicated_optimize2;
CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r1', d, k, 8192);
CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r2', d, k, 8192);
CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r1', d, k, 8192);
CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r2', d, k, 8192);
OPTIMIZE TABLE replicated_optimize1 FINAL;

View File

@ -8,14 +8,14 @@ CREATE TABLE zero_rows_per_granule1 (
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0;
CREATE TABLE zero_rows_per_granule2 (
p Date,
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0;
INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
@ -68,14 +68,14 @@ CREATE TABLE four_rows_per_granule1 (
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
CREATE TABLE four_rows_per_granule2 (
p Date,
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0;
INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
@ -135,14 +135,14 @@ CREATE TABLE adaptive_granularity_alter1 (
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
CREATE TABLE adaptive_granularity_alter2 (
p Date,
k UInt64,
v1 UInt64,
v2 Int64
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0;
INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);

View File

@ -1,3 +0,0 @@
200
400
CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,3 @@
200
400
CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192

View File

@ -1,9 +1,9 @@
DROP TABLE IF EXISTS ttl_repl1;
DROP TABLE IF EXISTS ttl_repl2;
CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '1')
CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '1')
PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY;
CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '2')
CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '2')
PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY;
INSERT INTO TABLE ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 100);

View File

@ -6,14 +6,14 @@ CREATE TABLE replicated_constraints1
a UInt32,
b UInt32,
CONSTRAINT a_constraint CHECK a < 10
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r1') ORDER BY (a);
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r1') ORDER BY (a);
CREATE TABLE replicated_constraints2
(
a UInt32,
b UInt32,
CONSTRAINT a_constraint CHECK a < 10
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r2') ORDER BY (a);
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r2') ORDER BY (a);
INSERT INTO replicated_constraints1 VALUES (1, 2);
INSERT INTO replicated_constraints2 VALUES (3, 4);

View File

@ -13,7 +13,7 @@ DROP TABLE IF EXISTS mt_without_pk;
DROP TABLE IF EXISTS replicated_mt_without_pk;
CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple();
CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple();
INSERT INTO replicated_mt_without_pk VALUES (1, 2);

View File

@ -3,9 +3,9 @@ DROP TABLE IF EXISTS table_for_synchronous_mutations2;
SELECT 'Replicated';
CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '1') ORDER BY k;
CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '1') ORDER BY k;
CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '2') ORDER BY k;
CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '2') ORDER BY k;
INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000);

View File

@ -6,7 +6,7 @@ CREATE TABLE test_alter_on_mutation
key UInt64,
value String
)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01062/alter_on_mutation', '1')
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/alter_on_mutation', '1')
ORDER BY key PARTITION BY date;
INSERT INTO test_alter_on_mutation select toDate('2020-01-05'), number, toString(number) from system.numbers limit 100;
@ -58,7 +58,7 @@ DROP TABLE IF EXISTS test_alter_on_mutation;
DROP TABLE IF EXISTS nested_alter;
CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01062/nested_alter', 'r2', d, k, 8192);
CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/nested_alter', 'r2', d, k, 8192);
INSERT INTO nested_alter VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');

View File

@ -1,11 +1,11 @@
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
1000
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
1000
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
10000
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192

View File

@ -5,7 +5,7 @@ CREATE TABLE alter_default
date Date,
key UInt64
)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01079/alter_default', '1')
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01079/alter_default', '1')
ORDER BY key;
INSERT INTO alter_default select toDate('2020-01-05'), number from system.numbers limit 100;

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS mutations_and_quorum1;
DROP TABLE IF EXISTS mutations_and_quorum2;
CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something);
CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something);
CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something);
CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something);
SET insert_quorum=2, insert_quorum_parallel=0;

View File

@ -1,2 +1,2 @@
4
CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192

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