Merge remote-tracking branch 'ck/master' into use_minmax_index

This commit is contained in:
taiyang-li 2022-03-30 11:23:24 +08:00
commit fcfabf4a0c
94 changed files with 2743 additions and 218 deletions

View File

@ -1,9 +1,12 @@
if (APPLE OR NOT ARCH_AMD64 OR SANITIZE STREQUAL "undefined")
# During cross-compilation in our CI we have to use llvm-tblgen and other building tools
# tools to be build for host architecture and everything else for target architecture (e.g. AArch64)
# Possible workaround is to use llvm-tblgen from some package...
# But lets just enable LLVM for native builds
if (CMAKE_CROSSCOMPILING OR SANITIZE STREQUAL "undefined")
set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF)
else()
set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON)
endif()
option (ENABLE_EMBEDDED_COMPILER "Enable support for 'compile_expressions' option for query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT})
if (NOT ENABLE_EMBEDDED_COMPILER)

View File

@ -1378,7 +1378,7 @@ $REF_SHA $SHA_TO_TEST $(numactl --hardware | sed -n 's/^available:[[:space:]]\+/
EOF
# Also insert some data about the check into the CI checks table.
"${client[@]}" --query "INSERT INTO "'"'"gh-data"'"'".checks FORMAT TSVWithNamesAndTypes" \
"${client[@]}" --query "INSERT INTO "'"'"default"'"'".checks FORMAT TSVWithNamesAndTypes" \
< ci-checks.tsv
set -x

View File

@ -8,7 +8,7 @@ toc_title: "版本折叠MergeTree"
这个引擎:
- 允许快速写入不断变化的对象状态。
- 删除后台中的旧对象状态。 这显降低了存储体积。
- 删除后台中的旧对象状态。 这显降低了存储体积。
请参阅部分 [崩溃](#table_engines_versionedcollapsingmergetree) 有关详细信息。

View File

@ -184,6 +184,11 @@ void LocalServer::tryInitPath()
if (path.back() != '/')
path += '/';
fs::create_directories(fs::path(path) / "user_defined/");
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(fs::path(path) / "metadata_dropped/");
global_context->setPath(path);
global_context->setTemporaryStorage(path + "tmp");
@ -565,7 +570,6 @@ void LocalServer::processConfig()
/// Lock path directory before read
status.emplace(fs::path(path) / "status", StatusFile::write_full_info);
fs::create_directories(fs::path(path) / "user_defined/");
LOG_DEBUG(log, "Loading user defined objects from {}", path);
Poco::File(path + "user_defined/").createDirectories();
UserDefinedSQLObjectsLoader::instance().loadObjects(global_context);
@ -573,9 +577,6 @@ void LocalServer::processConfig()
LOG_DEBUG(log, "Loaded user defined objects.");
LOG_DEBUG(log, "Loading metadata from {}", path);
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
loadMetadataSystem(global_context);
attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));

View File

@ -829,6 +829,36 @@ if (ThreadFuzzer::instance().isEffective())
fs::create_directories(path / "metadata_dropped/");
}
#if USE_ROCKSDB
/// Initialize merge tree metadata cache
if (config().has("merge_tree_metadata_cache"))
{
fs::create_directories(path / "rocksdb/");
size_t size = config().getUInt64("merge_tree_metadata_cache.lru_cache_size", 256 << 20);
bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false);
try
{
LOG_DEBUG(
log, "Initiailizing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted);
global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size);
}
catch (...)
{
if (continue_if_corrupted)
{
/// Rename rocksdb directory and reinitialize merge tree metadata cache
time_t now = time(nullptr);
fs::rename(path / "rocksdb", path / ("rocksdb.old." + std::to_string(now)));
global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size);
}
else
{
throw;
}
}
}
#endif
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);

View File

@ -1294,4 +1294,10 @@
</tables>
</rocksdb>
-->
<!-- Uncomment if enable merge tree metadata cache -->
<merge_tree_metadata_cache>
<lru_cache_size>268435456</lru_cache_size>
<continue_if_corrupted>true</continue_if_corrupted>
</merge_tree_metadata_cache>
</clickhouse>

View File

@ -494,6 +494,11 @@ endif()
target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::fast_float)
if (USE_ORC)
dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES})
dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${CMAKE_BINARY_DIR}/contrib/orc/c++/include")
endif ()
if (TARGET ch_contrib::rocksdb)
dbms_target_link_libraries(PUBLIC ch_contrib::rocksdb)
endif()

View File

@ -285,6 +285,13 @@
\
M(MainConfigLoads, "Number of times the main configuration was reloaded.") \
\
M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \
M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \
M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \
\
M(ScalarSubqueriesGlobalCacheHit, "Number of times a read from a scalar subquery was done using the global cache") \
M(ScalarSubqueriesLocalCacheHit, "Number of times a read from a scalar subquery was done using the local cache") \
M(ScalarSubqueriesCacheMiss, "Number of times a read from a scalar subquery was not cached and had to be calculated completely")

View File

@ -1,7 +1,18 @@
#include "gtest_global_context.h"
const ContextHolder & getContext()
{
return getMutableContext();
}
ContextHolder & getMutableContext()
{
static ContextHolder holder;
return holder;
}
void destroyContext()
{
auto & holder = getMutableContext();
return holder.destroy();
}

View File

@ -16,6 +16,17 @@ struct ContextHolder
}
ContextHolder(ContextHolder &&) = default;
void destroy()
{
context->shutdown();
context.reset();
shared_context.reset();
}
};
const ContextHolder & getContext();
ContextHolder & getMutableContext();
void destroyContext();

View File

@ -187,5 +187,4 @@ DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparin
DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule)
DECLARE_SETTING_ENUM_WITH_RENAME(MsgPackUUIDRepresentation, FormatSettings::MsgPackUUIDRepresentation)
}

View File

@ -9,6 +9,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <Common/JSONParsers/SimdJSONParser.h>
#include <Common/JSONParsers/RapidJSONParser.h>
#include <Common/JSONParsers/DummyJSONParser.h>
@ -158,22 +159,37 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field)
{
auto object = field.getObject();
DataTypePtr value_type;
bool is_object = false;
for (const auto key_value_pair : object)
{
auto type = getDataTypeFromJSONFieldImpl(key_value_pair.second);
if (!type)
return nullptr;
continue;
if (value_type && value_type->getName() != type->getName())
return nullptr;
value_type = type;
if (isObject(type))
{
is_object = true;
break;
}
if (!value_type)
return nullptr;
{
value_type = type;
}
else if (!value_type->equals(*type))
{
is_object = true;
break;
}
}
if (is_object)
return std::make_shared<DataTypeObject>("json", false);
if (value_type)
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
return nullptr;
}
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"};

View File

@ -7,6 +7,8 @@
#include <Formats/ReadSchemaUtils.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Common/assert_cast.h>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
namespace DB
{
@ -17,6 +19,28 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
static std::optional<NamesAndTypesList> getOrderedColumnsList(
const NamesAndTypesList & columns_list, const Names & columns_order_hint)
{
if (columns_list.size() != columns_order_hint.size())
return {};
std::unordered_map<String, DataTypePtr> available_columns;
for (const auto & [name, type] : columns_list)
available_columns.emplace(name, type);
NamesAndTypesList res;
for (const auto & name : columns_order_hint)
{
auto it = available_columns.find(name);
if (it == available_columns.end())
return {};
res.emplace_back(name, it->second);
}
return res;
}
ColumnsDescription readSchemaFromFormat(
const String & format_name,
const std::optional<FormatSettings> & format_settings,
@ -52,6 +76,22 @@ ColumnsDescription readSchemaFromFormat(
{
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message());
}
/// If we have "INSERT SELECT" query then try to order
/// columns as they are ordered in table schema for formats
/// without strict column order (like JSON and TSKV).
/// It will allow to execute simple data loading with query
/// "INSERT INTO table SELECT * FROM ..."
const auto & insertion_table = context->getInsertionTable();
if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty())
{
auto storage = DatabaseCatalog::instance().getTable(insertion_table, context);
auto metadata = storage->getInMemoryMetadataPtr();
auto names_in_storage = metadata->getColumns().getNamesOfPhysical();
auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage);
if (ordered_list)
names_and_types = *ordered_list;
}
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference", format_name);

View File

@ -53,6 +53,7 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnLowCardinality.h>
#include <Interpreters/Context.h>
#include <Common/HashTable/HashMap.h>
namespace DB
@ -3140,31 +3141,30 @@ private:
}
}
WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_type) const
WrapperType createTupleToObjectWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const
{
if (const auto * from_tuple = checkAndGetDataType<DataTypeTuple>(from_type.get()))
{
if (!from_tuple->haveExplicitNames())
if (!from_tuple.haveExplicitNames())
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_type->getName());
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_tuple.getName());
PathsInData paths;
DataTypes from_types;
std::tie(paths, from_types) = flattenTuple(from_type);
std::tie(paths, from_types) = flattenTuple(from_tuple.getPtr());
auto to_types = from_types;
for (auto & type : to_types)
{
if (isTuple(type) || isNested(type))
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_type->getName());
"Cast to Object can be performed only from flatten Named Tuple. Got: {}",
from_tuple.getName());
type = recursiveRemoveLowCardinality(type);
}
return [element_wrappers = getElementWrappers(from_types, to_types),
has_nullable_subcolumns = to_type->hasNullableSubcolumns(), from_types, to_types, paths]
has_nullable_subcolumns, from_types, to_types, paths]
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count)
{
size_t tuple_size = to_types.size();
@ -3187,6 +3187,93 @@ private:
return res;
};
}
WrapperType createMapToObjectWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const
{
auto key_value_types = from_map.getKeyValueTypes();
if (!isStringOrFixedString(key_value_types[0]))
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Cast to Object from Map can be performed only from Map "
"with String or FixedString key. Got: {}", from_map.getName());
const auto & value_type = key_value_types[1];
auto to_value_type = value_type;
if (!has_nullable_subcolumns && value_type->isNullable())
to_value_type = removeNullable(value_type);
if (has_nullable_subcolumns && !value_type->isNullable())
to_value_type = makeNullable(value_type);
DataTypes to_key_value_types{std::make_shared<DataTypeString>(), std::move(to_value_type)};
auto element_wrappers = getElementWrappers(key_value_types, to_key_value_types);
return [has_nullable_subcolumns, element_wrappers, key_value_types, to_key_value_types]
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t) -> ColumnPtr
{
const auto & column_map = assert_cast<const ColumnMap &>(*arguments.front().column);
const auto & offsets = column_map.getNestedColumn().getOffsets();
auto key_value_columns = column_map.getNestedData().getColumnsCopy();
for (size_t i = 0; i < 2; ++i)
{
ColumnsWithTypeAndName element{{key_value_columns[i], key_value_types[i], ""}};
key_value_columns[i] = element_wrappers[i](element, to_key_value_types[i], nullable_source, key_value_columns[i]->size());
}
const auto & key_column_str = assert_cast<const ColumnString &>(*key_value_columns[0]);
const auto & value_column = *key_value_columns[1];
using SubcolumnsMap = HashMap<StringRef, MutableColumnPtr, StringRefHash>;
SubcolumnsMap subcolumns;
for (size_t row = 0; row < offsets.size(); ++row)
{
for (size_t i = offsets[static_cast<ssize_t>(row) - 1]; i < offsets[row]; ++i)
{
auto ref = key_column_str.getDataAt(i);
bool inserted;
SubcolumnsMap::LookupResult it;
subcolumns.emplace(ref, it, inserted);
auto & subcolumn = it->getMapped();
if (inserted)
subcolumn = value_column.cloneEmpty()->cloneResized(row);
/// Map can have duplicated keys. We insert only first one.
if (subcolumn->size() == row)
subcolumn->insertFrom(value_column, i);
}
/// Insert default values for keys missed in current row.
for (const auto & [_, subcolumn] : subcolumns)
if (subcolumn->size() == row)
subcolumn->insertDefault();
}
auto column_object = ColumnObject::create(has_nullable_subcolumns);
for (auto && [key, subcolumn] : subcolumns)
{
PathInData path(key.toView());
column_object->addSubcolumn(path, std::move(subcolumn));
}
return column_object;
};
}
WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_type) const
{
if (const auto * from_tuple = checkAndGetDataType<DataTypeTuple>(from_type.get()))
{
return createTupleToObjectWrapper(*from_tuple, to_type->hasNullableSubcolumns());
}
else if (const auto * from_map = checkAndGetDataType<DataTypeMap>(from_type.get()))
{
return createMapToObjectWrapper(*from_map, to_type->hasNullableSubcolumns());
}
else if (checkAndGetDataType<DataTypeString>(from_type.get()))
{
return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count)
@ -3199,7 +3286,7 @@ private:
}
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Cast to Object can be performed only from flatten named tuple or string. Got: {}", from_type->getName());
"Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName());
}
template <typename FieldType>

View File

@ -0,0 +1,68 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/ObjectUtils.h>
#include <Columns/ColumnTuple.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
class FunctionFlattenTuple : public IFunction
{
public:
static constexpr auto name = "flattenTuple";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionFlattenTuple>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto & type = arguments[0];
const auto * type_tuple = checkAndGetDataType<DataTypeTuple>(type.get());
if (!type_tuple || !type_tuple->haveExplicitNames())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument for function '{}' must be Named Tuple. Got '{}'",
getName(), type->getName());
auto [paths, types] = flattenTuple(type);
Names names;
names.reserve(paths.size());
for (const auto & path : paths)
names.push_back(path.getPath());
return std::make_shared<DataTypeTuple>(types, names);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
auto column = arguments.at(0).column;
if (!checkAndGetColumn<ColumnTuple>(column.get()))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}. Expected ColumnTuple",
column->getName(), getName());
return flattenTuple(column);
}
};
}
void registerFunctionFlattenTuple(FunctionFactory & factory)
{
factory.registerFunction<FunctionFlattenTuple>();
}
}

View File

@ -80,6 +80,7 @@ void registerFunctionInitialQueryID(FunctionFactory & factory);
void registerFunctionServerUUID(FunctionFactory &);
void registerFunctionZooKeeperSessionUptime(FunctionFactory &);
void registerFunctionGetOSKernelVersion(FunctionFactory &);
void registerFunctionFlattenTuple(FunctionFactory &);
#if USE_ICU
void registerFunctionConvertCharset(FunctionFactory &);
@ -166,6 +167,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionServerUUID(factory);
registerFunctionZooKeeperSessionUptime(factory);
registerFunctionGetOSKernelVersion(factory);
registerFunctionFlattenTuple(factory);
#if USE_ICU
registerFunctionConvertCharset(factory);

View File

@ -15,4 +15,11 @@ public:
explicit ReadBufferFromString(std::string_view s) : ReadBufferFromMemory(s.data(), s.size()) {}
};
class ReadBufferFromOwnString : public String, public ReadBufferFromString
{
public:
explicit ReadBufferFromOwnString(const String & s_): String(s_), ReadBufferFromString(*this) {}
};
}

View File

@ -2,18 +2,15 @@
#include <iostream>
#include <base/types.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <base/types.h>
int main(int, char **)
int readAndPrint(DB::ReadBuffer & in)
{
try
{
std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'";
DB::ReadBufferFromString in(s);
DB::Int64 a;
DB::Float64 b;
DB::String c, d;
@ -31,12 +28,32 @@ int main(int, char **)
std::cout << a << ' ' << b << ' ' << c << '\t' << '\'' << d << '\'' << std::endl;
std::cout << in.count() << std::endl;
return 0;
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}
}
int main(int, char **)
{
{
std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'";
DB::ReadBufferFromString in(s);
if (readAndPrint(in))
std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl;
}
std::shared_ptr<DB::ReadBufferFromOwnString> in;
{
std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'";
in = std::make_shared<DB::ReadBufferFromOwnString>(s);
}
if (readAndPrint(*in))
std::cout << "readAndPrint from ReadBufferFromOwnString failed" << std::endl;
return 0;
}

View File

@ -12,6 +12,7 @@
#include <Storages/MarkCache.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <IO/UncompressedCache.h>
#include <IO/MMappedFileCache.h>
#include <IO/ReadHelpers.h>
@ -607,6 +608,15 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
}
}
#if USE_ROCKSDB
{
if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache())
{
new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys();
}
}
#endif
#if USE_EMBEDDED_COMPILER
{
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
@ -617,6 +627,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
}
#endif
new_values["Uptime"] = getContext()->getUptimeSeconds();
/// Process process memory usage according to OS

View File

@ -85,11 +85,15 @@
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <Interpreters/SynonymsExtensions.h>
#include <Interpreters/Lemmatizers.h>
#include <Interpreters/ClusterDiscovery.h>
#include <filesystem>
#if USE_ROCKSDB
#include <rocksdb/table.h>
#endif
namespace fs = std::filesystem;
@ -276,6 +280,11 @@ struct ContextSharedPart
Context::ConfigReloadCallback config_reload_callback;
#if USE_ROCKSDB
/// Global merge tree metadata cache, stored in rocksdb.
MergeTreeMetadataCachePtr merge_tree_metadata_cache;
#endif
ContextSharedPart()
: access_control(std::make_unique<AccessControl>())
, global_overcommit_tracker(&process_list)
@ -410,6 +419,15 @@ struct ContextSharedPart
trace_collector.reset();
/// Stop zookeeper connection
zookeeper.reset();
#if USE_ROCKSDB
/// Shutdown merge tree metadata cache
if (merge_tree_metadata_cache)
{
merge_tree_metadata_cache->shutdown();
merge_tree_metadata_cache.reset();
}
#endif
}
/// Can be removed w/o context lock
@ -2048,6 +2066,23 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const
return zookeeper->second;
}
#if USE_ROCKSDB
MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const
{
auto cache = tryGetMergeTreeMetadataCache();
if (!cache)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart");
return cache;
}
MergeTreeMetadataCachePtr Context::tryGetMergeTreeMetadataCache() const
{
return shared->merge_tree_metadata_cache;
}
#endif
void Context::resetZooKeeper() const
{
std::lock_guard lock(shared->zookeeper_mutex);
@ -2291,6 +2326,13 @@ void Context::initializeTraceCollector()
shared->initializeTraceCollector(getTraceLog());
}
#if USE_ROCKSDB
void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size)
{
shared->merge_tree_metadata_cache = MergeTreeMetadataCache::create(dir, size);
}
#endif
bool Context::hasTraceCollector() const
{
return shared->hasTraceCollector();

View File

@ -16,6 +16,7 @@
#include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include "config_core.h"
#include <boost/container/flat_set.hpp>
@ -152,6 +153,12 @@ using ReadTaskCallback = std::function<String()>;
using MergeTreeReadTaskCallback = std::function<std::optional<PartitionReadResponse>(PartitionReadRequest)>;
#if USE_ROCKSDB
class MergeTreeMetadataCache;
using MergeTreeMetadataCachePtr = std::shared_ptr<MergeTreeMetadataCache>;
#endif
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
struct IHostContext
@ -179,6 +186,7 @@ private:
std::unique_ptr<ContextSharedPart> shared;
};
/** A set of known objects that can be used in the query.
* Consists of a shared part (always common to all sessions and queries)
* and copied part (which can be its own for each session or query).
@ -680,6 +688,11 @@ public:
UInt32 getZooKeeperSessionUptime() const;
#if USE_ROCKSDB
MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const;
MergeTreeMetadataCachePtr tryGetMergeTreeMetadataCache() const;
#endif
#if USE_NURAFT
std::shared_ptr<KeeperDispatcher> & getKeeperDispatcher() const;
#endif
@ -769,6 +782,10 @@ public:
/// Call after initialization before using trace collector.
void initializeTraceCollector();
#if USE_ROCKSDB
void initializeMergeTreeMetadataCache(const String & dir, size_t size);
#endif
bool hasTraceCollector() const;
/// Nullptr if the query log is not ready for this moment.

View File

@ -358,6 +358,7 @@ BlockIO InterpreterInsertQuery::execute()
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
new_context->setInsertionTable(getContext()->getInsertionTable());
InterpreterSelectWithUnionQuery interpreter_select{
query.select, new_context, SelectQueryOptions(QueryProcessingStage::Complete, 1)};

View File

@ -657,6 +657,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
}
if (const auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
{
/// Save insertion table (not table function). TODO: support remote() table function.
auto table_id = insert_interpreter->getDatabaseTable();
if (!table_id.empty())
context->setInsertionTable(std::move(table_id));
}
{
std::unique_ptr<OpenTelemetrySpanHolder> span;
if (context->query_trace_context.trace_id != UUID())
@ -667,14 +675,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
res = interpreter->execute();
}
if (const auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
{
/// Save insertion table (not table function). TODO: support remote() table function.
auto table_id = insert_interpreter->getDatabaseTable();
if (!table_id.empty())
context->setInsertionTable(std::move(table_id));
}
}
if (process_list_entry)

View File

@ -18,6 +18,10 @@ public:
virtual NamesAndTypesList readSchema() = 0;
/// True if order of columns is important in format.
/// Exceptions: JSON, TSKV.
virtual bool hasStrictOrderOfColumns() const { return true; }
virtual ~ISchemaReader() = default;
protected:
@ -60,6 +64,7 @@ class IRowWithNamesSchemaReader : public ISchemaReader
public:
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
NamesAndTypesList readSchema() override;
bool hasStrictOrderOfColumns() const override { return false; }
protected:
/// Read one row and determine types of columns in it.

View File

@ -45,7 +45,8 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & context)
static MergeTreeReaderSettings getMergeTreeReaderSettings(
const ContextPtr & context, const SelectQueryInfo & query_info)
{
const auto & settings = context->getSettingsRef();
return
@ -53,6 +54,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & con
.read_settings = context->getReadSettings(),
.save_marks_in_cache = true,
.checksum_on_read = settings.checksum_on_read,
.read_in_order = query_info.input_order_info != nullptr,
};
}
@ -82,7 +84,7 @@ ReadFromMergeTree::ReadFromMergeTree(
getPrewhereInfo(query_info_),
data_.getPartitionValueType(),
virt_column_names_)})
, reader_settings(getMergeTreeReaderSettings(context_))
, reader_settings(getMergeTreeReaderSettings(context_, query_info_))
, prepared_parts(std::move(parts_))
, real_column_names(std::move(real_column_names_))
, virt_column_names(std::move(virt_column_names_))
@ -206,6 +208,7 @@ ProcessorPtr ReadFromMergeTree::createSource(
.colums_to_read = required_columns
};
}
return std::make_shared<TSource>(
data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes,
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
@ -921,7 +924,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
total_marks_pk += part->index_granularity.getMarksCountWithoutFinal();
parts_before_pk = parts.size();
auto reader_settings = getMergeTreeReaderSettings(context);
auto reader_settings = getMergeTreeReaderSettings(context, query_info);
bool use_skip_indexes = settings.use_skip_indexes;
if (select.final() && !settings.use_skip_indexes_if_final)

View File

@ -1,9 +1,11 @@
#include "IMergeTreeDataPart.h"
#include <optional>
#include <boost/algorithm/string/join.hpp>
#include <string_view>
#include <Core/Defines.h>
#include <IO/HashingWriteBuffer.h>
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -11,6 +13,8 @@
#include <Storages/MergeTree/localBackup.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
#include <Storages/MergeTree/PartMetadataManagerWithCache.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/ZooKeeper/ZooKeeper.h>
@ -61,13 +65,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
{
size_t file_size = disk->getFileSize(path);
return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size);
}
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path)
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager)
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
const auto & partition_key = metadata_snapshot->getPartitionKey();
@ -79,8 +77,8 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis
hyperrectangle.reserve(minmax_idx_size);
for (size_t i = 0; i < minmax_idx_size; ++i)
{
String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx");
auto file = openForReading(disk_, file_name);
String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx";
auto file = manager->read(file_name);
auto serialization = minmax_column_types[i]->getDefaultSerialization();
Field min_val;
@ -192,6 +190,19 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other)
}
}
void IMergeTreeDataPart::MinMaxIndex::appendFiles(const MergeTreeData & data, Strings & files)
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
size_t minmax_idx_size = minmax_column_names.size();
for (size_t i = 0; i < minmax_idx_size; ++i)
{
String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx";
files.push_back(file_name);
}
}
static void incrementStateMetric(IMergeTreeDataPart::State state)
{
@ -299,6 +310,7 @@ IMergeTreeDataPart::IMergeTreeDataPart(
, index_granularity_info(storage_, part_type_)
, part_type(part_type_)
, parent_part(parent_part_)
, use_metadata_cache(storage.use_metadata_cache)
{
if (parent_part)
state = State::Active;
@ -306,6 +318,8 @@ IMergeTreeDataPart::IMergeTreeDataPart(
incrementTypeMetric(part_type);
minmax_idx = std::make_shared<MinMaxIndex>();
initializePartMetadataManager();
}
IMergeTreeDataPart::IMergeTreeDataPart(
@ -324,6 +338,7 @@ IMergeTreeDataPart::IMergeTreeDataPart(
, index_granularity_info(storage_, part_type_)
, part_type(part_type_)
, parent_part(parent_part_)
, use_metadata_cache(storage.use_metadata_cache)
{
if (parent_part)
state = State::Active;
@ -331,6 +346,8 @@ IMergeTreeDataPart::IMergeTreeDataPart(
incrementTypeMetric(part_type);
minmax_idx = std::make_shared<MinMaxIndex>();
initializePartMetadataManager();
}
IMergeTreeDataPart::~IMergeTreeDataPart()
@ -558,9 +575,10 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
return checksum->second.file_size;
}
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageSnapshotPtr & storage_snapshot) const
{
const auto & storage_columns = metadata_snapshot->getColumns().getAllPhysical();
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withSubcolumns();
auto storage_columns = storage_snapshot->getColumns(options);
MergeTreeData::AlterConversions alter_conversions;
if (!parent_part)
alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
@ -637,6 +655,33 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
loadDefaultCompressionCodec();
}
void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection) const
{
if (isStoredOnDisk())
{
appendFilesOfUUID(files);
appendFilesOfColumns(files);
appendFilesOfChecksums(files);
appendFilesOfIndexGranularity(files);
appendFilesOfIndex(files);
appendFilesOfRowsCount(files);
appendFilesOfPartitionAndMinMaxIndex(files);
appendFilesOfTTLInfos(files);
appendFilesOfDefaultCompressionCodec(files);
}
if (!parent_part && include_projection)
{
for (const auto & [projection_name, projection_part] : projection_parts)
{
Strings projection_files;
projection_part->appendFilesOfColumnsChecksumsIndexes(projection_files, true);
for (const auto & projection_file : projection_files)
files.push_back(fs::path(projection_part->relative_path) / projection_file);
}
}
}
void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency)
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
@ -657,6 +702,11 @@ void IMergeTreeDataPart::loadIndexGranularity()
throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Currently we don't cache mark files of part, because cache other meta files is enough to speed up loading.
void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) const
{
}
void IMergeTreeDataPart::loadIndex()
{
/// It can be empty in case of mutations
@ -680,9 +730,9 @@ void IMergeTreeDataPart::loadIndex()
loaded_index[i]->reserve(index_granularity.getMarksCount());
}
String index_path = fs::path(getFullRelativePath()) / "primary.idx";
auto index_file = openForReading(volume->getDisk(), index_path);
String index_name = "primary.idx";
String index_path = fs::path(getFullRelativePath()) / index_name;
auto index_file = metadata_manager->read(index_name);
size_t marks_count = index_granularity.getMarksCount();
Serializations key_serializations(key_size);
@ -709,6 +759,19 @@ void IMergeTreeDataPart::loadIndex()
}
}
void IMergeTreeDataPart::appendFilesOfIndex(Strings & files) const
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part)
metadata_snapshot = metadata_snapshot->projections.has(name) ? metadata_snapshot->projections.get(name).metadata : nullptr;
if (!metadata_snapshot)
return;
if (metadata_snapshot->hasPrimaryKey())
files.push_back("primary.idx");
}
NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const
{
if (!isStoredOnDisk())
@ -733,14 +796,14 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec()
}
String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME;
if (!volume->getDisk()->exists(path))
bool exists = metadata_manager->exists(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
if (!exists)
{
default_codec = detectDefaultCompressionCodec();
}
else
{
auto file_buf = openForReading(volume->getDisk(), path);
auto file_buf = metadata_manager->read(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
String codec_line;
readEscapedStringUntilEOL(codec_line, *file_buf);
@ -748,7 +811,13 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec()
if (!checkString("CODEC", buf))
{
LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced automatically, from data on disk", name, path, codec_line);
LOG_WARNING(
storage.log,
"Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced "
"automatically, from data on disk",
name,
path,
codec_line);
default_codec = detectDefaultCompressionCodec();
}
@ -766,6 +835,11 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec()
}
}
void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files)
{
files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
}
CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const
{
/// In memory parts doesn't have any compression
@ -828,7 +902,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
{
String path = getFullRelativePath();
if (!parent_part)
partition.load(storage, volume->getDisk(), path);
partition.load(storage, metadata_manager);
if (!isEmpty())
{
@ -836,7 +910,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
// projection parts don't have minmax_idx, and it's always initialized
minmax_idx->initialized = true;
else
minmax_idx->load(storage, volume->getDisk(), path);
minmax_idx->load(storage, metadata_manager);
}
if (parent_part)
return;
@ -851,13 +925,26 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
ErrorCodes::CORRUPTED_DATA);
}
void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) const
{
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part)
return;
if (!parent_part)
partition.appendFiles(storage, files);
if (!isEmpty())
if (!parent_part)
minmax_idx->appendFiles(storage, files);
}
void IMergeTreeDataPart::loadChecksums(bool require)
{
const String path = fs::path(getFullRelativePath()) / "checksums.txt";
if (volume->getDisk()->exists(path))
bool exists = metadata_manager->exists("checksums.txt");
if (exists)
{
auto buf = openForReading(volume->getDisk(), path);
auto buf = metadata_manager->read("checksums.txt");
if (checksums.read(*buf))
{
assertEOF(*buf);
@ -888,13 +975,18 @@ void IMergeTreeDataPart::loadChecksums(bool require)
}
}
void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files)
{
files.push_back("checksums.txt");
}
void IMergeTreeDataPart::loadRowsCount()
{
String path = fs::path(getFullRelativePath()) / "count.txt";
auto read_rows_count = [&]()
{
auto buf = openForReading(volume->getDisk(), path);
auto buf = metadata_manager->read("count.txt");
readIntText(rows_count, *buf);
assertEOF(*buf);
};
@ -905,7 +997,8 @@ void IMergeTreeDataPart::loadRowsCount()
}
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part)
{
if (!volume->getDisk()->exists(path))
bool exists = metadata_manager->exists("count.txt");
if (!exists)
throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
read_rows_count();
@ -1003,12 +1096,17 @@ void IMergeTreeDataPart::loadRowsCount()
}
}
void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files)
{
files.push_back("count.txt");
}
void IMergeTreeDataPart::loadTTLInfos()
{
String path = fs::path(getFullRelativePath()) / "ttl.txt";
if (volume->getDisk()->exists(path))
bool exists = metadata_manager->exists("ttl.txt");
if (exists)
{
auto in = openForReading(volume->getDisk(), path);
auto in = metadata_manager->read("ttl.txt");
assertString("ttl format version: ", *in);
size_t format_version;
readText(format_version, *in);
@ -1030,19 +1128,29 @@ void IMergeTreeDataPart::loadTTLInfos()
}
}
void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files)
{
files.push_back("ttl.txt");
}
void IMergeTreeDataPart::loadUUID()
{
String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME;
if (volume->getDisk()->exists(path))
bool exists = metadata_manager->exists(UUID_FILE_NAME);
if (exists)
{
auto in = openForReading(volume->getDisk(), path);
auto in = metadata_manager->read(UUID_FILE_NAME);
readText(uuid, *in);
if (uuid == UUIDHelpers::Nil)
throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR);
}
}
void IMergeTreeDataPart::appendFilesOfUUID(Strings & files)
{
files.push_back(UUID_FILE_NAME);
}
void IMergeTreeDataPart::loadColumns(bool require)
{
String path = fs::path(getFullRelativePath()) / "columns.txt";
@ -1051,7 +1159,8 @@ void IMergeTreeDataPart::loadColumns(bool require)
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
NamesAndTypesList loaded_columns;
if (!volume->getDisk()->exists(path))
bool exists = metadata_manager->exists("columns.txt");
if (!exists)
{
/// We can get list of columns only from columns.txt in compact parts.
if (require || part_type == Type::COMPACT)
@ -1074,7 +1183,8 @@ void IMergeTreeDataPart::loadColumns(bool require)
}
else
{
loaded_columns.readText(*volume->getDisk()->readFile(path));
auto in = metadata_manager->read("columns.txt");
loaded_columns.readText(*in);
for (const auto & column : loaded_columns)
{
@ -1091,14 +1201,23 @@ void IMergeTreeDataPart::loadColumns(bool require)
};
SerializationInfoByName infos(loaded_columns, settings);
path = getFullRelativePath() + SERIALIZATION_FILE_NAME;
if (volume->getDisk()->exists(path))
infos.readJSON(*volume->getDisk()->readFile(path));
exists = metadata_manager->exists(SERIALIZATION_FILE_NAME);
if (exists)
{
auto in = metadata_manager->read(SERIALIZATION_FILE_NAME);
infos.readJSON(*in);
}
setColumns(loaded_columns);
setSerializationInfos(infos);
}
void IMergeTreeDataPart::appendFilesOfColumns(Strings & files)
{
files.push_back("columns.txt");
files.push_back(SERIALIZATION_FILE_NAME);
}
bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const
{
/// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds
@ -1150,9 +1269,12 @@ try
}
}
metadata_manager->deleteAll(true);
metadata_manager->assertAllDeleted(true);
volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr)));
volume->getDisk()->moveDirectory(from, to);
relative_path = new_relative_path;
metadata_manager->updateAll(true);
SyncGuardPtr sync_guard;
if (storage.getSettings()->fsync_part_directory)
@ -1190,6 +1312,18 @@ std::optional<bool> IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const
return !storage.unlockSharedData(*this);
}
void IMergeTreeDataPart::initializePartMetadataManager()
{
#if USE_ROCKSDB
if (use_metadata_cache)
metadata_manager = std::make_shared<PartMetadataManagerWithCache>(this, storage.getContext()->getMergeTreeMetadataCache());
else
metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(this);
#else
metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(this);
#endif
}
void IMergeTreeDataPart::remove() const
{
std::optional<bool> keep_shared_data = keepSharedDataInDecoupledStorage();
@ -1209,6 +1343,9 @@ void IMergeTreeDataPart::remove() const
return;
}
metadata_manager->deleteAll(false);
metadata_manager->assertAllDeleted(false);
/** Atomic directory removal:
* - rename directory to temporary name;
* - remove it recursive.
@ -1314,6 +1451,9 @@ void IMergeTreeDataPart::remove() const
void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const
{
metadata_manager->deleteAll(false);
metadata_manager->assertAllDeleted(false);
String to = fs::path(parent_to) / relative_path;
auto disk = volume->getDisk();
if (checksums.empty())
@ -1661,6 +1801,35 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const
return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
}
IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const
{
assert(use_metadata_cache);
String file_name = std::filesystem::path(file_path).filename();
const auto filenames_without_checksums = getFileNamesWithoutChecksums();
auto it = checksums.files.find(file_name);
if (filenames_without_checksums.count(file_name) == 0 && it != checksums.files.end())
{
return it->second.file_hash;
}
if (!volume->getDisk()->exists(file_path))
{
return {};
}
std::unique_ptr<ReadBufferFromFileBase> in_file = volume->getDisk()->readFile(file_path);
HashingReadBuffer in_hash(*in_file);
String value;
readStringUntilEOF(value, in_hash);
return in_hash.getHash();
}
std::unordered_map<String, IMergeTreeDataPart::uint128> IMergeTreeDataPart::checkMetadata() const
{
return metadata_manager->check();
}
bool isCompactPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT);

View File

@ -14,6 +14,7 @@
#include <Storages/MergeTree/MergeTreeIOSettings.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <DataTypes/Serializations/SerializationInfo.h>
#include <Storages/MergeTree/IPartMetadataManager.h>
#include <shared_mutex>
@ -60,6 +61,8 @@ public:
using Type = MergeTreeDataPartType;
using uint128 = IPartMetadataManager::uint128;
IMergeTreeDataPart(
const MergeTreeData & storage_,
@ -148,6 +151,7 @@ public:
/// Initialize columns (from columns.txt if exists, or create from column files if not).
/// Load checksums from checksums.txt if exists. Load index if required.
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency);
void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const;
String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; }
@ -164,7 +168,7 @@ public:
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
/// If no checksums are present returns the name of the first physically existing column.
String getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const;
String getColumnNameWithMinimumCompressedSize(const StorageSnapshotPtr & storage_snapshot) const;
bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); }
@ -243,7 +247,7 @@ public:
using TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos;
TTLInfos ttl_infos;
mutable TTLInfos ttl_infos;
/// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex
void setState(State new_state) const;
@ -300,14 +304,16 @@ public:
{
}
void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager);
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path);
[[nodiscard]] WrittenFiles store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
[[nodiscard]] WrittenFiles store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
void update(const Block & block, const Names & column_names);
void merge(const MinMaxIndex & other);
static void appendFiles(const MergeTreeData & data, Strings & files);
};
using MinMaxIndexPtr = std::shared_ptr<MinMaxIndex>;
@ -429,6 +435,12 @@ public:
/// Required for distinguish different copies of the same part on remote FS.
String getUniqueId() const;
/// Get checksums of metadata file in part directory
IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const;
/// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true)
std::unordered_map<String, uint128> checkMetadata() const;
protected:
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
@ -455,6 +467,11 @@ protected:
std::map<String, std::shared_ptr<IMergeTreeDataPart>> projection_parts;
/// Disabled when USE_ROCKSDB is OFF or use_metadata_cache is set to false in merge tree settings
bool use_metadata_cache = false;
mutable PartMetadataManagerPtr metadata_manager;
void removeIfNeeded();
virtual void checkConsistency(bool require_part_metadata) const;
@ -468,6 +485,9 @@ protected:
std::optional<bool> keepSharedDataInDecoupledStorage() const;
void initializePartMetadataManager();
private:
/// In compact parts order of columns is necessary
NameToNumber column_name_to_position;
@ -478,36 +498,54 @@ private:
/// Reads part unique identifier (if exists) from uuid.txt
void loadUUID();
static void appendFilesOfUUID(Strings & files);
/// Reads columns names and types from columns.txt
void loadColumns(bool require);
static void appendFilesOfColumns(Strings & files);
/// If checksums.txt exists, reads file's checksums (and sizes) from it
void loadChecksums(bool require);
static void appendFilesOfChecksums(Strings & files);
/// Loads marks index granularity into memory
virtual void loadIndexGranularity();
virtual void appendFilesOfIndexGranularity(Strings & files) const;
/// Loads index file.
void loadIndex();
void appendFilesOfIndex(Strings & files) const;
/// Load rows count for this part from disk (for the newer storage format version).
/// For the older format version calculates rows count from the size of a column with a fixed size.
void loadRowsCount();
static void appendFilesOfRowsCount(Strings & files);
/// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros
void loadTTLInfos();
static void appendFilesOfTTLInfos(Strings & files);
void loadPartitionAndMinMaxIndex();
void calculateColumnsSizesOnDisk();
void calculateSecondaryIndicesSizesOnDisk();
void appendFilesOfPartitionAndMinMaxIndex(Strings & files) const;
/// Load default compression codec from file default_compression_codec.txt
/// if it not exists tries to deduce codec from compressed column without
/// any specifial compression.
void loadDefaultCompressionCodec();
static void appendFilesOfDefaultCompressionCodec(Strings & files);
/// Found column without specific compression and return codec
/// for this column with default parameters.
CompressionCodecPtr detectDefaultCompressionCodec() const;

View File

@ -0,0 +1,11 @@
#include "IPartMetadataManager.h"
#include <Disks/IVolume.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
namespace DB
{
IPartMetadataManager::IPartMetadataManager(const IMergeTreeDataPart * part_) : part(part_), disk(part->volume->getDisk())
{
}
}

View File

@ -0,0 +1,59 @@
#pragma once
#include <unordered_map>
#include <city.h>
#include <base/types.h>
namespace DB
{
class IMergeTreeDataPart;
class SeekableReadBuffer;
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Interface for managing metadata of merge tree part.
/// IPartMetadataManager has two implementations:
/// - PartMetadataManagerOrdinary: manage metadata from disk directly. deleteAll/assertAllDeleted/updateAll/check
/// are all empty implementations because they are not needed for PartMetadataManagerOrdinary(those operations
/// are done implicitly when removing or renaming part directory).
/// - PartMetadataManagerWithCache: manage metadata from RocksDB cache and disk.
class IPartMetadataManager
{
public:
using uint128 = CityHash_v1_0_2::uint128;
explicit IPartMetadataManager(const IMergeTreeDataPart * part_);
virtual ~IPartMetadataManager() = default;
/// Read metadata content and return SeekableReadBuffer object.
virtual std::unique_ptr<SeekableReadBuffer> read(const String & file_name) const = 0;
/// Return true if metadata exists in part.
virtual bool exists(const String & file_name) const = 0;
/// Delete all metadatas in part.
/// If include_projection is true, also delete metadatas in projection parts.
virtual void deleteAll(bool include_projection) = 0;
/// Assert that all metadatas in part are deleted.
/// If include_projection is true, also assert that all metadatas in projection parts are deleted.
virtual void assertAllDeleted(bool include_projection) const = 0;
/// Update all metadatas in part.
/// If include_projection is true, also update metadatas in projection parts.
virtual void updateAll(bool include_projection) = 0;
/// Check all metadatas in part.
virtual std::unordered_map<String, uint128> check() const = 0;
protected:
const IMergeTreeDataPart * part;
const DiskPtr disk;
};
using PartMetadataManagerPtr = std::shared_ptr<IPartMetadataManager>;
}

View File

@ -53,7 +53,7 @@ String Range::toString() const
/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string.
static String extractFixedPrefixFromLikePattern(const String & like_pattern)
String extractFixedPrefixFromLikePattern(const String & like_pattern)
{
String fixed_prefix;

View File

@ -442,4 +442,6 @@ private:
bool strict;
};
String extractFixedPrefixFromLikePattern(const String & like_pattern);
}

View File

@ -24,7 +24,7 @@ namespace
/// least one existing (physical) column in part.
bool injectRequiredColumnsRecursively(
const String & column_name,
const ColumnsDescription & storage_columns,
const StorageSnapshotPtr & storage_snapshot,
const MergeTreeData::AlterConversions & alter_conversions,
const MergeTreeData::DataPartPtr & part,
Names & columns,
@ -36,7 +36,8 @@ bool injectRequiredColumnsRecursively(
/// stages.
checkStackSize();
auto column_in_storage = storage_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, column_name);
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns().withExtendedObjects();
auto column_in_storage = storage_snapshot->tryGetColumn(options, column_name);
if (column_in_storage)
{
auto column_name_in_part = column_in_storage->getNameInStorage();
@ -63,7 +64,8 @@ bool injectRequiredColumnsRecursively(
/// Column doesn't have default value and don't exist in part
/// don't need to add to required set.
const auto column_default = storage_columns.getDefault(column_name);
auto metadata_snapshot = storage_snapshot->getMetadataForQuery();
const auto column_default = metadata_snapshot->getColumns().getDefault(column_name);
if (!column_default)
return false;
@ -73,39 +75,36 @@ bool injectRequiredColumnsRecursively(
bool result = false;
for (const auto & identifier : identifiers)
result |= injectRequiredColumnsRecursively(identifier, storage_columns, alter_conversions, part, columns, required_columns, injected_columns);
result |= injectRequiredColumnsRecursively(identifier, storage_snapshot, alter_conversions, part, columns, required_columns, injected_columns);
return result;
}
}
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns)
NameSet injectRequiredColumns(
const MergeTreeData & storage,
const StorageSnapshotPtr & storage_snapshot,
const MergeTreeData::DataPartPtr & part,
Names & columns)
{
NameSet required_columns{std::begin(columns), std::end(columns)};
NameSet injected_columns;
bool have_at_least_one_physical_column = false;
const auto & storage_columns = metadata_snapshot->getColumns();
MergeTreeData::AlterConversions alter_conversions;
if (!part->isProjectionPart())
alter_conversions = storage.getAlterConversionsForPart(part);
for (size_t i = 0; i < columns.size(); ++i)
{
auto name_in_storage = Nested::extractTableName(columns[i]);
if (storage_columns.has(name_in_storage) && isObject(storage_columns.get(name_in_storage).type))
{
have_at_least_one_physical_column = true;
continue;
}
/// We are going to fetch only physical columns
if (!storage_columns.hasColumnOrSubcolumn(GetColumnsOptions::AllPhysical, columns[i]))
throw Exception("There is no physical column or subcolumn " + columns[i] + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns().withExtendedObjects();
if (!storage_snapshot->tryGetColumn(options, columns[i]))
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no physical column or subcolumn {} in table", columns[i]);
have_at_least_one_physical_column |= injectRequiredColumnsRecursively(
columns[i], storage_columns, alter_conversions,
columns[i], storage_snapshot, alter_conversions,
part, columns, required_columns, injected_columns);
}
@ -115,7 +114,7 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada
*/
if (!have_at_least_one_physical_column)
{
const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(metadata_snapshot);
const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(storage_snapshot);
columns.push_back(minimum_size_column_name);
/// correctly report added column
injected_columns.insert(columns.back());
@ -271,7 +270,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
Names pre_column_names;
/// inject columns required for defaults evaluation
bool should_reorder = !injectRequiredColumns(storage, storage_snapshot->getMetadataForQuery(), data_part, column_names).empty();
bool should_reorder = !injectRequiredColumns(storage, storage_snapshot, data_part, column_names).empty();
if (prewhere_info)
{
@ -296,7 +295,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
if (pre_column_names.empty())
pre_column_names.push_back(column_names[0]);
const auto injected_pre_columns = injectRequiredColumns(storage, storage_snapshot->getMetadataForQuery(), data_part, pre_column_names);
const auto injected_pre_columns = injectRequiredColumns(storage, storage_snapshot, data_part, pre_column_names);
if (!injected_pre_columns.empty())
should_reorder = true;

View File

@ -22,7 +22,7 @@ using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredict
* so that you can calculate the DEFAULT expression for these columns.
* Adds them to the `columns`.
*/
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns);
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns);
/// A batch of work for MergeTreeThreadSelectBlockInputStream

View File

@ -214,6 +214,7 @@ MergeTreeData::MergeTreeData(
, parts_mover(this)
, background_operations_assignee(*this, BackgroundJobsAssignee::Type::DataProcessing, getContext())
, background_moves_assignee(*this, BackgroundJobsAssignee::Type::Moving, getContext())
, use_metadata_cache(getSettings()->use_metadata_cache)
{
context_->getGlobalContext()->initializeBackgroundExecutorsIfNeeded();
@ -333,6 +334,11 @@ MergeTreeData::MergeTreeData(
LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', "
"'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason);
#if !USE_ROCKSDB
if (use_metadata_cache)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb");
#endif
common_assignee_trigger = [this] (bool delay) noexcept
{
if (delay)
@ -1372,7 +1378,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size());
}
/// Is the part directory old.
/// True if its modification time and the modification time of all files inside it is less then threshold.
/// (Only files on the first level of nesting are considered).

View File

@ -946,6 +946,7 @@ protected:
friend class StorageReplicatedMergeTree;
friend class MergeTreeDataWriter;
friend class MergeTask;
friend class IPartMetadataManager;
bool require_part_metadata;
@ -1028,6 +1029,7 @@ protected:
/// And for ReplicatedMergeTree we don't have LogEntry type for this operation.
BackgroundJobsAssignee background_operations_assignee;
BackgroundJobsAssignee background_moves_assignee;
bool use_metadata_cache;
/// Strongly connected with two fields above.
/// Every task that is finished will ask to assign a new one into an executor.

View File

@ -69,6 +69,7 @@ private:
ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set<String> * processed_substreams) const;
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
};
}

View File

@ -877,12 +877,22 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
{
std::atomic<size_t> total_rows{0};
/// Do not check number of read rows if we have reading
/// in order of sorting key with limit.
/// In general case, when there exists WHERE clause
/// it's impossible to estimate number of rows precisely,
/// because we can stop reading at any time.
SizeLimits limits;
if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read)
if (settings.read_overflow_mode == OverflowMode::THROW
&& settings.max_rows_to_read
&& !query_info.input_order_info)
limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode);
SizeLimits leaf_limits;
if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf)
if (settings.read_overflow_mode_leaf == OverflowMode::THROW
&& settings.max_rows_to_read_leaf
&& !query_info.input_order_info)
leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf);
auto mark_cache = context->getIndexMarkCache();

View File

@ -20,6 +20,8 @@ struct MergeTreeReaderSettings
bool save_marks_in_cache = false;
/// Validate checksums on reading (should be always enabled in production).
bool checksum_on_read = true;
/// True if we read in order of sorting key.
bool read_in_order = false;
};
struct MergeTreeWriterSettings

View File

@ -0,0 +1,107 @@
#include "MergeTreeMetadataCache.h"
#if USE_ROCKSDB
#include <Common/ProfileEvents.h>
#include <base/logger_useful.h>
namespace ProfileEvents
{
extern const Event MergeTreeMetadataCachePut;
extern const Event MergeTreeMetadataCacheGet;
extern const Event MergeTreeMetadataCacheDelete;
extern const Event MergeTreeMetadataCacheSeek;
}
namespace DB
{
namespace ErrorCodes
{
extern const int SYSTEM_ERROR;
}
std::unique_ptr<MergeTreeMetadataCache> MergeTreeMetadataCache::create(const String & dir, size_t size)
{
assert(size != 0);
rocksdb::Options options;
rocksdb::BlockBasedTableOptions table_options;
rocksdb::DB * db;
options.create_if_missing = true;
auto cache = rocksdb::NewLRUCache(size);
table_options.block_cache = cache;
options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options));
rocksdb::Status status = rocksdb::DB::Open(options, dir, &db);
if (status != rocksdb::Status::OK())
throw Exception(
ErrorCodes::SYSTEM_ERROR,
"Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).",
dir,
status.ToString());
return std::make_unique<MergeTreeMetadataCache>(db);
}
MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value)
{
auto options = rocksdb::WriteOptions();
options.sync = true;
options.disableWAL = false;
auto status = rocksdb->Put(options, key, value);
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut);
return status;
}
MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key)
{
auto options = rocksdb::WriteOptions();
options.sync = true;
options.disableWAL = false;
auto status = rocksdb->Delete(options, key);
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete);
LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString());
return status;
}
MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value)
{
auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value);
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet);
LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString());
return status;
}
void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values)
{
auto * it = rocksdb->NewIterator(rocksdb::ReadOptions());
rocksdb::Slice target(prefix);
for (it->Seek(target); it->Valid(); it->Next())
{
const auto key = it->key();
if (!key.starts_with(target))
break;
const auto value = it->value();
keys.emplace_back(key.data(), key.size());
values.emplace_back(value.data(), value.size());
}
LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size());
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek);
delete it;
}
uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const
{
uint64_t keys = 0;
rocksdb->GetAggregatedIntProperty("rocksdb.estimate-num-keys", &keys);
return keys;
}
void MergeTreeMetadataCache::shutdown()
{
rocksdb->Close();
rocksdb.reset();
}
}
#endif

View File

@ -0,0 +1,45 @@
#pragma once
#include "config_core.h"
#if USE_ROCKSDB
#include <base/types.h>
#include <Core/Types.h>
#include <Poco/Logger.h>
#include <rocksdb/table.h>
#include <rocksdb/db.h>
namespace DB
{
class MergeTreeMetadataCache
{
public:
using Status = rocksdb::Status;
static std::unique_ptr<MergeTreeMetadataCache> create(const String & dir, size_t size);
explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_}
{
assert(rocksdb);
}
MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete;
MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete;
Status put(const String & key, const String & value);
Status del(const String & key);
Status get(const String & key, String & value);
void getByPrefix(const String & prefix, Strings & keys, Strings & values);
uint64_t getEstimateNumKeys() const;
void shutdown();
private:
std::unique_ptr<rocksdb::DB> rocksdb;
Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache");
};
using MergeTreeMetadataCachePtr = std::shared_ptr<MergeTreeMetadataCache>;
}
#endif

View File

@ -178,12 +178,6 @@ namespace
};
}
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
{
size_t file_size = disk->getFileSize(path);
return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size);
}
String MergeTreePartition::getID(const MergeTreeData & storage) const
{
return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block);
@ -373,15 +367,15 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe
}
}
void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path)
void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager)
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (!metadata_snapshot->hasPartitionKey())
return;
const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block;
auto partition_file_path = part_path + "partition.dat";
auto file = openForReading(disk, partition_file_path);
auto file = manager->read("partition.dat");
value.resize(partition_key_sample.columns());
for (size_t i = 0; i < partition_key_sample.columns(); ++i)
partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file);
@ -402,7 +396,9 @@ std::unique_ptr<WriteBufferFromFileBase> MergeTreePartition::store(const Block &
auto out = disk->writeFile(part_path + "partition.dat");
HashingWriteBuffer out_hashing(*out);
for (size_t i = 0; i < value.size(); ++i)
{
partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing);
}
out_hashing.next();
checksums.files["partition.dat"].file_size = out_hashing.count();
@ -462,4 +458,14 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr &
return partition_key;
}
void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files)
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (!metadata_snapshot->hasPartitionKey())
return;
files.push_back("partition.dat");
}
}

View File

@ -4,6 +4,7 @@
#include <Disks/IDisk.h>
#include <IO/WriteBuffer.h>
#include <Storages/KeyDescription.h>
#include <Storages/MergeTree/IPartMetadataManager.h>
#include <Core/Field.h>
namespace DB
@ -37,7 +38,8 @@ public:
void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const;
void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path);
void load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager);
/// Store functions return write buffer with written but not finalized data.
/// User must call finish() for returned object.
[[nodiscard]] std::unique_ptr<WriteBufferFromFileBase> store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
@ -47,6 +49,8 @@ public:
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context);
static void appendFiles(const MergeTreeData & storage, Strings & files);
/// Adjust partition key and execute its expression on block. Return sample block according to used expression.
static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context);

View File

@ -39,9 +39,12 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
{
/// Actually it means that parallel reading from replicas enabled
/// and we have to collaborate with initiator.
/// In this case we won't set approximate rows, because it will be accounted multiple times
if (!extension_.has_value())
/// In this case we won't set approximate rows, because it will be accounted multiple times.
/// Also do not count amount of read rows if we read in order of sorting key,
/// because we don't know actual amount of read rows in case when limit is set.
if (!extension_.has_value() && !reader_settings.read_in_order)
addTotalRowsApprox(total_rows);
ordered_names = header_without_virtual_columns.getNames();
}

View File

@ -41,7 +41,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
addTotalRowsApprox(data_part->rows_count);
/// Add columns because we don't want to read empty blocks
injectRequiredColumns(storage, storage_snapshot->metadata, data_part, columns_to_read);
injectRequiredColumns(storage, storage_snapshot, data_part, columns_to_read);
NamesAndTypesList columns_for_reader;
if (take_column_types_from_storage)
{

View File

@ -139,6 +139,7 @@ struct Settings;
/** Experimental/work in progress feature. Unsafe for production. */ \
M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \
M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \
M(Bool, use_metadata_cache, false, "Experimental feature to speed up parts loading process by using MergeTree metadata cache", 0) \
\
/** Obsolete settings. Kept for backward compatibility only. */ \
M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \

View File

@ -0,0 +1,33 @@
#include "PartMetadataManagerOrdinary.h"
#include <IO/ReadBufferFromFileBase.h>
#include <Disks/IDisk.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
namespace DB
{
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
{
size_t file_size = disk->getFileSize(path);
return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size);
}
PartMetadataManagerOrdinary::PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_) : IPartMetadataManager(part_)
{
}
std::unique_ptr<SeekableReadBuffer> PartMetadataManagerOrdinary::read(const String & file_name) const
{
String file_path = fs::path(part->getFullRelativePath()) / file_name;
return openForReading(disk, file_path);
}
bool PartMetadataManagerOrdinary::exists(const String & file_name) const
{
return disk->exists(fs::path(part->getFullRelativePath()) / file_name);
}
}

View File

@ -0,0 +1,29 @@
#pragma once
#include <Storages/MergeTree/IPartMetadataManager.h>
namespace DB
{
class PartMetadataManagerOrdinary : public IPartMetadataManager
{
public:
explicit PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_);
~PartMetadataManagerOrdinary() override = default;
std::unique_ptr<SeekableReadBuffer> read(const String & file_name) const override;
bool exists(const String & file_name) const override;
void deleteAll(bool /*include_projection*/) override {}
void assertAllDeleted(bool /*include_projection*/) const override {}
void updateAll(bool /*include_projection*/) override {}
std::unordered_map<String, uint128> check() const override { return {}; }
};
}

View File

@ -0,0 +1,273 @@
#include "PartMetadataManagerWithCache.h"
#if USE_ROCKSDB
#include <Common/hex.h>
#include <Common/ErrorCodes.h>
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
namespace ProfileEvents
{
extern const Event MergeTreeMetadataCacheHit;
extern const Event MergeTreeMetadataCacheMiss;
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int CORRUPTED_DATA;
extern const int NO_SUCH_PROJECTION_IN_TABLE;
}
PartMetadataManagerWithCache::PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_)
: IPartMetadataManager(part_), cache(cache_)
{
}
String PartMetadataManagerWithCache::getKeyFromFilePath(const String & file_path) const
{
return disk->getName() + ":" + file_path;
}
String PartMetadataManagerWithCache::getFilePathFromKey(const String & key) const
{
return key.substr(disk->getName().size() + 1);
}
std::unique_ptr<SeekableReadBuffer> PartMetadataManagerWithCache::read(const String & file_name) const
{
String file_path = fs::path(part->getFullRelativePath()) / file_name;
String key = getKeyFromFilePath(file_path);
String value;
auto status = cache->get(key, value);
if (!status.ok())
{
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss);
auto in = disk->readFile(file_path);
readStringUntilEOF(value, *in);
cache->put(key, value);
}
else
{
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit);
}
return std::make_unique<ReadBufferFromOwnString>(value);
}
bool PartMetadataManagerWithCache::exists(const String & file_name) const
{
String file_path = fs::path(part->getFullRelativePath()) / file_name;
String key = getKeyFromFilePath(file_path);
String value;
auto status = cache->get(key, value);
if (status.ok())
{
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit);
return true;
}
else
{
ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss);
return disk->exists(fs::path(part->getFullRelativePath()) / file_name);
}
}
void PartMetadataManagerWithCache::deleteAll(bool include_projection)
{
Strings file_names;
part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection);
String value;
for (const auto & file_name : file_names)
{
String file_path = fs::path(part->getFullRelativePath()) / file_name;
String key = getKeyFromFilePath(file_path);
auto status = cache->del(key);
if (!status.ok())
{
status = cache->get(key, value);
if (status.IsNotFound())
continue;
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"deleteAll failed include_projection:{} status:{}, file_path:{}",
include_projection,
status.ToString(),
file_path);
}
}
}
void PartMetadataManagerWithCache::updateAll(bool include_projection)
{
Strings file_names;
part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection);
String value;
String read_value;
for (const auto & file_name : file_names)
{
String file_path = fs::path(part->getFullRelativePath()) / file_name;
if (!disk->exists(file_path))
continue;
auto in = disk->readFile(file_path);
readStringUntilEOF(value, *in);
String key = getKeyFromFilePath(file_path);
auto status = cache->put(key, value);
if (!status.ok())
{
status = cache->get(key, read_value);
if (status.IsNotFound() || read_value == value)
continue;
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"updateAll failed include_projection:{} status:{}, file_path:{}",
include_projection,
status.ToString(),
file_path);
}
}
}
void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const
{
Strings keys;
std::vector<uint128> _;
getKeysAndCheckSums(keys, _);
if (keys.empty())
return;
String file_path;
String file_name;
for (const auto & key : keys)
{
file_path = getFilePathFromKey(key);
file_name = fs::path(file_path).filename();
/// Metadata file belongs to current part
if (fs::path(part->getFullRelativePath()) / file_name == file_path)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Data part {} with type {} with meta file {} still in cache",
part->name,
part->getType().toString(),
file_path);
/// File belongs to projection part of current part
if (!part->isProjectionPart() && include_projection)
{
const auto & projection_parts = part->getProjectionParts();
for (const auto & [projection_name, projection_part] : projection_parts)
{
if (fs::path(projection_part->getFullRelativePath()) / file_name == file_path)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Data part {} with type {} with meta file {} with projection name still in cache",
part->name,
part->getType().toString(),
file_path,
projection_name);
}
}
}
}
}
void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vector<uint128> & checksums) const
{
String prefix = getKeyFromFilePath(fs::path(part->getFullRelativePath()) / "");
Strings values;
cache->getByPrefix(prefix, keys, values);
size_t size = keys.size();
for (size_t i = 0; i < size; ++i)
{
ReadBufferFromString rbuf(values[i]);
HashingReadBuffer hbuf(rbuf);
checksums.push_back(hbuf.getHash());
}
}
std::unordered_map<String, IPartMetadataManager::uint128> PartMetadataManagerWithCache::check() const
{
/// Only applies for normal part stored on disk
if (part->isProjectionPart() || !part->isStoredOnDisk())
return {};
/// The directory of projection part is under the directory of its parent part
const auto filenames_without_checksums = part->getFileNamesWithoutChecksums();
std::unordered_map<String, uint128> results;
Strings keys;
std::vector<uint128> cache_checksums;
std::vector<uint128> disk_checksums;
getKeysAndCheckSums(keys, cache_checksums);
for (size_t i = 0; i < keys.size(); ++i)
{
const auto & key = keys[i];
String file_path = getFilePathFromKey(key);
String file_name = fs::path(file_path).filename();
results.emplace(file_name, cache_checksums[i]);
/// File belongs to normal part
if (fs::path(part->getFullRelativePath()) / file_name == file_path)
{
auto disk_checksum = part->getActualChecksumByFile(file_path);
if (disk_checksum != cache_checksums[i])
throw Exception(
ErrorCodes::CORRUPTED_DATA,
"Checksums doesn't match in part {}. Expected: {}. Found {}.",
part->name,
getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second),
getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second));
disk_checksums.push_back(disk_checksum);
continue;
}
/// File belongs to projection part
String proj_dir_name = fs::path(file_path).parent_path().filename();
auto pos = proj_dir_name.find_last_of('.');
if (pos == String::npos)
{
throw Exception(
ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE,
"There is no projection in part: {} contains file: {} with directory name: {}",
part->name,
file_path,
proj_dir_name);
}
String proj_name = proj_dir_name.substr(0, pos);
const auto & projection_parts = part->getProjectionParts();
auto it = projection_parts.find(proj_name);
if (it == projection_parts.end())
{
throw Exception(
ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE,
"There is no projection {} in part: {} contains file: {}",
proj_name, part->name, file_path);
}
auto disk_checksum = it->second->getActualChecksumByFile(file_path);
if (disk_checksum != cache_checksums[i])
throw Exception(
ErrorCodes::CORRUPTED_DATA,
"Checksums doesn't match in projection part {} {}. Expected: {}. Found {}.",
part->name, proj_name,
getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second),
getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second));
disk_checksums.push_back(disk_checksum);
}
return results;
}
}
#endif

View File

@ -0,0 +1,56 @@
#pragma once
#include "config_core.h"
#if USE_ROCKSDB
#include <Storages/MergeTree/IPartMetadataManager.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
namespace DB
{
/// PartMetadataManagerWithCache stores metadatas of part in RocksDB as cache layer to speed up
/// loading process of merge tree table.
class PartMetadataManagerWithCache : public IPartMetadataManager
{
public:
PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_);
~PartMetadataManagerWithCache() override = default;
/// First read the metadata from RocksDB cache, then from disk.
std::unique_ptr<SeekableReadBuffer> read(const String & file_name) const override;
/// First judge existence of the metadata in RocksDB cache, then in disk.
bool exists(const String & file_name) const override;
/// Delete all metadatas in part from RocksDB cache.
void deleteAll(bool include_projection) override;
/// Assert all metadatas in part from RocksDB cache are deleted.
void assertAllDeleted(bool include_projection) const override;
/// Update all metadatas in part from RocksDB cache.
/// Need to be called after part directory is renamed.
void updateAll(bool include_projection) override;
/// Check if all metadatas in part from RocksDB cache are up to date.
std::unordered_map<String, uint128> check() const override;
private:
/// Get cache key from path of metadata file.
/// Format: <disk_name>:relative/full/path/of/metadata/file
String getKeyFromFilePath(const String & file_path) const;
/// Get metadata file path from cache key.
String getFilePathFromKey(const String & key) const;
/// Get cache keys and checksums of corresponding metadata in a part(including projection parts)
void getKeysAndCheckSums(Strings & keys, std::vector<uint128> & checksums) const;
MergeTreeMetadataCachePtr cache;
};
}
#endif

View File

@ -399,6 +399,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na
LOG_WARNING(log, "We have part {} covering part {}", part->name, part_name);
}
part->checkMetadata();
return {part_name, true, ""};
}

View File

@ -0,0 +1,83 @@
#include "config_core.h"
#if USE_ROCKSDB
#include <gtest/gtest.h>
#include <rocksdb/table.h>
#include <rocksdb/db.h>
#include <Interpreters/Context.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
using namespace DB;
class MergeTreeMetadataCacheTest : public ::testing::Test
{
public:
void SetUp() override
{
cache = MergeTreeMetadataCache::create("./db/", 268435456);
}
void TearDown() override
{
cache->shutdown();
cache.reset();
}
MergeTreeMetadataCachePtr cache;
};
TEST_F(MergeTreeMetadataCacheTest, testCommon)
{
std::vector<String> files
= {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"};
String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/";
for (const auto & file : files)
{
auto status = cache->put(prefix + file, prefix + file);
ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk);
}
for (const auto & file : files)
{
String value;
auto status = cache->get(prefix + file, value);
ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk);
ASSERT_EQ(value, prefix + file);
}
{
Strings keys;
Strings values;
cache->getByPrefix(prefix, keys, values);
ASSERT_EQ(keys.size(), files.size());
ASSERT_EQ(values.size(), files.size());
for (size_t i = 0; i < files.size(); ++i)
{
ASSERT_EQ(values[i], keys[i]);
}
}
for (const auto & file : files)
{
auto status = cache->del(prefix + file);
ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk);
}
for (const auto & file : files)
{
String value;
auto status = cache->get(prefix + file, value);
ASSERT_EQ(status.code(), rocksdb::Status::Code::kNotFound);
}
{
Strings keys;
Strings values;
cache->getByPrefix(prefix, keys, values);
ASSERT_EQ(keys.size(), 0);
ASSERT_EQ(values.size(), 0);
}
}
#endif

View File

@ -1616,6 +1616,8 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
auto out = disk->writeFile(tmp_checksums_path, 4096);
part->checksums.write(*out);
disk->moveFile(tmp_checksums_path, checksums_path);
part->checkMetadata();
results.emplace_back(part->name, true, "Checksums recounted and written to disk.");
}
catch (const Exception & ex)
@ -1632,6 +1634,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
try
{
checkDataPart(part, true);
part->checkMetadata();
results.emplace_back(part->name, true, "");
}
catch (const Exception & ex)

View File

@ -51,40 +51,42 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options)
NamesAndTypesList StorageSnapshot::getColumnsByNames(const GetColumnsOptions & options, const Names & names) const
{
NamesAndTypesList res;
const auto & columns = getMetadataForQuery()->getColumns();
for (const auto & name : names)
{
auto column = columns.tryGetColumn(options, name);
if (column && !isObject(column->type))
{
res.emplace_back(std::move(*column));
continue;
res.push_back(getColumn(options, name));
return res;
}
std::optional<NameAndTypePair> StorageSnapshot::tryGetColumn(const GetColumnsOptions & options, const String & column_name) const
{
const auto & columns = getMetadataForQuery()->getColumns();
auto column = columns.tryGetColumn(options, column_name);
if (column && (!isObject(column->type) || !options.with_extended_objects))
return column;
if (options.with_extended_objects)
{
auto object_column = object_columns.tryGetColumn(options, name);
auto object_column = object_columns.tryGetColumn(options, column_name);
if (object_column)
{
res.emplace_back(std::move(*object_column));
continue;
}
return object_column;
}
if (options.with_virtuals)
{
auto it = virtual_columns.find(name);
auto it = virtual_columns.find(column_name);
if (it != virtual_columns.end())
return NameAndTypePair(column_name, it->second);
}
return {};
}
NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, const String & column_name) const
{
res.emplace_back(name, it->second);
continue;
}
}
auto column = tryGetColumn(options, column_name);
if (!column)
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", column_name);
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", name);
}
return res;
return *column;
}
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const

View File

@ -61,6 +61,10 @@ struct StorageSnapshot
/// Get columns with types according to options only for requested names.
NamesAndTypesList getColumnsByNames(const GetColumnsOptions & options, const Names & names) const;
/// Get column with type according to options for requested name.
std::optional<NameAndTypePair> tryGetColumn(const GetColumnsOptions & options, const String & column_name) const;
NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const;
/// Block with ordinary + materialized + aliases + virtuals + subcolumns.
Block getSampleBlockForColumns(const Names & column_names) const;

View File

@ -560,6 +560,8 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData(
const std::optional<FormatSettings> & format_settings,
ContextPtr context)
{
context->getRemoteHostFilter().checkURL(Poco::URI(uri));
Poco::Net::HTTPBasicCredentials credentials;
std::vector<String> urls_to_check;

View File

@ -0,0 +1,143 @@
#include <Storages/System/StorageSystemMergeTreeMetadataCache.h>
#if USE_ROCKSDB
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
NamesAndTypesList StorageSystemMergeTreeMetadataCache::getNamesAndTypes()
{
return {
{"key", std::make_shared<DataTypeString>()},
{"value", std::make_shared<DataTypeString>()},
};
}
static bool extractKeyImpl(const IAST & elem, String & res, bool & precise)
{
const auto * function = elem.as<ASTFunction>();
if (!function)
return false;
if (function->name == "and")
{
for (const auto & child : function->arguments->children)
{
bool tmp_precise = false;
if (extractKeyImpl(*child, res, tmp_precise))
{
precise = tmp_precise;
return true;
}
}
return false;
}
if (function->name == "equals" || function->name == "like")
{
const auto & args = function->arguments->as<ASTExpressionList &>();
const IAST * value;
if (args.children.size() != 2)
return false;
const ASTIdentifier * ident;
if ((ident = args.children.at(0)->as<ASTIdentifier>()))
value = args.children.at(1).get();
else if ((ident = args.children.at(1)->as<ASTIdentifier>()))
value = args.children.at(0).get();
else
return false;
if (ident->name() != "key")
return false;
const auto * literal = value->as<ASTLiteral>();
if (!literal)
return false;
if (literal->value.getType() != Field::Types::String)
return false;
res = literal->value.safeGet<String>();
precise = function->name == "equals";
return true;
}
return false;
}
/// Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause.
static String extractKey(const ASTPtr & query, bool& precise)
{
const auto & select = query->as<ASTSelectQuery &>();
if (!select.where())
return "";
String res;
return extractKeyImpl(*select.where(), res, precise) ? res : "";
}
void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const
{
bool precise = false;
String key = extractKey(query_info.query, precise);
if (key.empty())
throw Exception(
"SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS);
auto cache = context->getMergeTreeMetadataCache();
if (precise)
{
String value;
if (cache->get(key, value) != MergeTreeMetadataCache::Status::OK())
return;
size_t col_num = 0;
res_columns[col_num++]->insert(key);
res_columns[col_num++]->insert(value);
}
else
{
String target = extractFixedPrefixFromLikePattern(key);
if (target.empty())
throw Exception(
"SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS);
Strings keys;
Strings values;
keys.reserve(4096);
values.reserve(4096);
cache->getByPrefix(target, keys, values);
if (keys.empty())
return;
assert(keys.size() == values.size());
for (size_t i = 0; i < keys.size(); ++i)
{
size_t col_num = 0;
res_columns[col_num++]->insert(keys[i]);
res_columns[col_num++]->insert(values[i]);
}
}
}
}
#endif

View File

@ -0,0 +1,32 @@
#pragma once
#include "config_core.h"
#if USE_ROCKSDB
#include <base/shared_ptr_helper.h>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class Context;
/// Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for testing purposes.
class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper<StorageSystemMergeTreeMetadataCache>, public IStorageSystemOneBlock<StorageSystemMergeTreeMetadataCache>
{
friend struct shared_ptr_helper<StorageSystemMergeTreeMetadataCache>;
public:
std::string getName() const override { return "SystemMergeTreeMetadataCache"; }
static NamesAndTypesList getNamesAndTypes();
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
};
}
#endif

View File

@ -75,6 +75,7 @@
#if USE_ROCKSDB
#include <Storages/RocksDB/StorageSystemRocksDB.h>
#include <Storages/System/StorageSystemMergeTreeMetadataCache.h>
#endif
@ -129,6 +130,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
#endif
#if USE_ROCKSDB
attach<StorageSystemRocksDB>(context, system_database, "rocksdb");
attach<StorageSystemMergeTreeMetadataCache>(context, system_database, "merge_tree_metadata_cache");
#endif
}

View File

@ -10,12 +10,12 @@ from get_robot_token import get_parameter_from_ssm
class ClickHouseHelper:
def __init__(self, url=None):
if url is None:
self.url = get_parameter_from_ssm("clickhouse-test-stat-url2")
url = get_parameter_from_ssm("clickhouse-test-stat-url")
self.url = url
self.auth = {
"X-ClickHouse-User": get_parameter_from_ssm(
"clickhouse-test-stat-login2"
),
"X-ClickHouse-Key": "",
"X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"),
"X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password")
}
@staticmethod
@ -179,7 +179,7 @@ def mark_flaky_tests(clickhouse_helper, check_name, test_results):
check_name=check_name
)
tests_data = clickhouse_helper.select_json_each_row("gh-data", query)
tests_data = clickhouse_helper.select_json_each_row("default", query)
master_failed_tests = {row["test_name"] for row in tests_data}
logging.info("Found flaky tests: %s", ", ".join(master_failed_tests))

View File

@ -197,4 +197,4 @@ if __name__ == "__main__":
report_url,
CHECK_NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -459,7 +459,7 @@ def main():
NAME,
)
ch_helper = ClickHouseHelper()
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if __name__ == "__main__":

View File

@ -234,7 +234,7 @@ def main():
NAME,
)
ch_helper = ClickHouseHelper()
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if __name__ == "__main__":

View File

@ -114,4 +114,4 @@ if __name__ == "__main__":
report_url,
NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -204,7 +204,7 @@ if __name__ == "__main__":
report_url,
NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
# Refuse other checks to run if fast test failed
if state != "success":

View File

@ -356,7 +356,7 @@ if __name__ == "__main__":
report_url,
check_name_with_group,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if state != "success":
if "force-tests" in pr_info.labels:

View File

@ -279,4 +279,4 @@ if __name__ == "__main__":
report_url,
check_name_with_group,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -271,5 +271,5 @@ if __name__ == "__main__":
report_url,
CHECK_NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
clear_autoscaling_group()

View File

@ -147,4 +147,4 @@ if __name__ == "__main__":
report_url,
CHECK_NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -176,4 +176,4 @@ if __name__ == "__main__":
report_url,
check_name,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -117,4 +117,4 @@ if __name__ == "__main__":
report_url,
NAME,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -173,4 +173,4 @@ if __name__ == "__main__":
report_url,
check_name,
)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)

View File

@ -276,3 +276,8 @@ def test_HDFS(start_cluster):
assert "not allowed" in node7.query_and_get_error(
"SELECT * FROM hdfs('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'word String')"
)
def test_schema_inference(start_cluster):
error = node7.query_and_get_error("desc url('http://test.com`, 'TSVRaw'')")
assert error.find("ReadWriteBufferFromHTTPBase") == -1

View File

@ -12,6 +12,8 @@ TABLE_NAME = "blob_storage_table"
CONTAINER_NAME = "cont"
CLUSTER_NAME = "test_cluster"
drop_table_statement = f"DROP TABLE {TABLE_NAME} ON CLUSTER {CLUSTER_NAME} SYNC"
@pytest.fixture(scope="module")
def cluster():
@ -55,7 +57,6 @@ def create_table(node, table_name, replica, **additional_settings):
ORDER BY id
SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}"""
node.query(f"DROP TABLE IF EXISTS {table_name}")
node.query(create_table_statement)
assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)"
@ -104,3 +105,4 @@ def test_zero_copy_replication(cluster):
)
assert get_large_objects_count(blob_container_client) == 2
node1.query(drop_table_statement)

View File

@ -90,6 +90,9 @@ def test_backup_from_old_version(started_cluster):
assert node1.query("CHECK TABLE dest_table") == "1\n"
node1.query("DROP TABLE source_table")
node1.query("DROP TABLE dest_table")
def test_backup_from_old_version_setting(started_cluster):
node2.query(
@ -137,6 +140,9 @@ def test_backup_from_old_version_setting(started_cluster):
assert node2.query("CHECK TABLE dest_table") == "1\n"
node2.query("DROP TABLE source_table")
node2.query("DROP TABLE dest_table")
def test_backup_from_old_version_config(started_cluster):
node3.query(
@ -190,6 +196,9 @@ def test_backup_from_old_version_config(started_cluster):
assert node3.query("CHECK TABLE dest_table") == "1\n"
node3.query("DROP TABLE source_table")
node3.query("DROP TABLE dest_table")
def test_backup_and_alter(started_cluster):
node4.query(
@ -223,3 +232,6 @@ def test_backup_and_alter(started_cluster):
assert node4.query("SELECT sum(A) FROM test.backup_table") == "2\n"
assert node4.query("SELECT B + 2 FROM test.backup_table") == "4\n"
node4.query("DROP TABLE test.backup_table")
node4.query("DROP DATABASE test")

View File

@ -94,7 +94,7 @@ def _check_exception(exception, expected_tries=3):
@pytest.fixture(scope="module", params=["configs", "configs_secure"])
def started_cluster(request):
cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__, request.param)
cluster.__with_ssl_config = request.param == "configs_secure"
main_configs = []
main_configs += [os.path.join(request.param, "config.d/remote_servers.xml")]

View File

@ -14,4 +14,8 @@
<table>part_log</table>
<flush_interval_milliseconds>500</flush_interval_milliseconds>
</part_log>
<merge_tree_metadata_cache>
<lru_cache_size>268435456</lru_cache_size>
<continue_if_corrupted>true</continue_if_corrupted>
</merge_tree_metadata_cache>
</clickhouse>

View File

@ -846,14 +846,15 @@ def get_paths_for_partition_from_part_log(node, table, partition_id):
@pytest.mark.parametrize(
"name,engine",
"name,engine,use_metadata_cache",
[
pytest.param("altering_mt", "MergeTree()", id="mt"),
pytest.param("altering_mt", "MergeTree()", "false", id="mt"),
pytest.param("altering_mt", "MergeTree()", "true", id="mt_use_metadata_cache"),
# ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",),
# SYSTEM STOP MERGES doesn't disable merges assignments
],
)
def test_alter_move(start_cluster, name, engine):
def test_alter_move(start_cluster, name, engine, use_metadata_cache):
try:
node1.query(
"""
@ -863,9 +864,9 @@ def test_alter_move(start_cluster, name, engine):
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy='jbods_with_external'
SETTINGS storage_policy='jbods_with_external', use_metadata_cache={use_metadata_cache}
""".format(
name=name, engine=engine
name=name, engine=engine, use_metadata_cache=use_metadata_cache
)
)
@ -875,6 +876,8 @@ def test_alter_move(start_cluster, name, engine):
node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 66)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name))
assert node1.query("CHECK TABLE " + name) == "1\n"
used_disks = get_used_disks_for_table(node1, name)
assert all(
d.startswith("jbod") for d in used_disks
@ -892,6 +895,7 @@ def test_alter_move(start_cluster, name, engine):
name, first_part
)
)
assert node1.query("CHECK TABLE " + name) == "1\n"
disk = node1.query(
"SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(
name, first_part
@ -906,6 +910,7 @@ def test_alter_move(start_cluster, name, engine):
node1.query(
"ALTER TABLE {} MOVE PART '{}' TO DISK 'jbod1'".format(name, first_part)
)
assert node1.query("CHECK TABLE " + name) == "1\n"
disk = node1.query(
"SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(
name, first_part
@ -920,6 +925,7 @@ def test_alter_move(start_cluster, name, engine):
node1.query(
"ALTER TABLE {} MOVE PARTITION 201904 TO VOLUME 'external'".format(name)
)
assert node1.query("CHECK TABLE " + name) == "1\n"
disks = (
node1.query(
"SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format(
@ -938,6 +944,7 @@ def test_alter_move(start_cluster, name, engine):
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO DISK 'jbod2'".format(name))
assert node1.query("CHECK TABLE " + name) == "1\n"
disks = (
node1.query(
"SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format(

View File

@ -18,7 +18,7 @@ DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
rm "${CURDIR}"/01037_point_data
@ -34,7 +34,7 @@ CREATE TABLE test_01037.polygons_array
ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
rm "${CURDIR}"/01037_polygon_data

View File

@ -12,7 +12,7 @@ function run_selects()
{
thread_num=$1
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables
WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper'
WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache'
AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num")
for t in "${tables_arr[@]}"

View File

@ -0,0 +1,672 @@
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (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);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1
TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;
CHECK TABLE test_metadata_cache.check_part_metadata_cache;
1

View File

@ -0,0 +1,101 @@
#!/usr/bin/env bash
# Tags: no-fasttest, long
# Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB.
# To suppress Warning messages from CHECK TABLE
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
table_engines=(ReplicatedMergeTree)
database_engines=(Ordinary Atomic)
use_metadata_caches=(false true)
use_projections=(false true)
use_compact_data_parts=(false true)
for table_engine in "${table_engines[@]}"; do
for database_engine in "${database_engines[@]}"; do
for use_metadata_cache in "${use_metadata_caches[@]}"; do
for use_projection in "${use_projections[@]}"; do
for use_compact_data_part in "${use_compact_data_parts[@]}"; do
echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}; use_compact_data_part:${use_compact_data_part}"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;"
${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};"
table_engine_clause=""
if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then
table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')"
elif [[ "$table_engine" == "MergeTree" ]]; then
table_engine_clause="ENGINE MergeTree()"
fi
projection_clause=""
if [[ "$use_projection" == "true" ]]; then
projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)"
fi
compact_data_part_clause=", min_bytes_for_wide_part = 10485760"
if [[ $use_compact_data_part == "true" ]]; then
compact_data_part_clause=", min_bytes_for_wide_part = 0"
fi
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache} ${compact_data_part_clause}"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Insert first batch of data.
${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (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);"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Insert second batch of data.
${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# First update.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Second update.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# First delete.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Second delete.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Insert third batch of data.
${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Drop one partition.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Add column.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Delete column.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Add TTL.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Modify TTL.
${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
# Truncate table.
${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;"
${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;"
done
done
done
done
done

View File

@ -0,0 +1,4 @@
800000 2000000 1400000 900000
800000 2000000 1400000 900000
Tuple(col0 UInt64, col1 UInt64, col2 UInt64, col3 UInt64, col4 UInt64, col5 UInt64, col6 UInt64, col7 UInt64, col8 UInt64)
1600000 4000000 2800000 1800000

View File

@ -0,0 +1,41 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS t_json;
DROP TABLE IF EXISTS t_map;
SET allow_experimental_object_type = 1;
CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id;
CREATE TABLE t_map(id UInt64, m Map(String, UInt64)) ENGINE = MergeTree ORDER BY id;
INSERT INTO t_map
SELECT
number,
(
arrayMap(x -> 'col' || toString(x), range(number % 10)),
range(number % 10)
)::Map(String, UInt64)
FROM numbers(1000000);
INSERT INTO t_json SELECT id, m FROM t_map;
SELECT sum(m['col1']), sum(m['col4']), sum(m['col7']), sum(m['col8'] = 0) FROM t_map;
SELECT sum(obj.col1), sum(obj.col4), sum(obj.col7), sum(obj.col8 = 0) FROM t_json;
SELECT toTypeName(obj) FROM t_json LIMIT 1;
INSERT INTO t_json
SELECT
number,
(
arrayMap(x -> 'col' || toString(x), range(number % 10)),
range(number % 10)
)::Map(FixedString(4), UInt64)
FROM numbers(1000000);
SELECT sum(obj.col1), sum(obj.col4), sum(obj.col7), sum(obj.col8 = 0) FROM t_json;
INSERT INTO t_json
SELECT number, (range(number % 10), range(number % 10))::Map(UInt64, UInt64)
FROM numbers(1000000); -- { serverError 53 }
DROP TABLE IF EXISTS t_json;
DROP TABLE IF EXISTS t_map;

View File

@ -0,0 +1,2 @@
Tuple(foo Int8, k1 Int8, k2 Int8)
1

View File

@ -0,0 +1,19 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS t_json;
SET allow_experimental_object_type = 1;
CREATE TABLE t_json(id UInt64, obj JSON)
ENGINE = MergeTree ORDER BY id
SETTINGS min_bytes_for_wide_part = 0;
SYSTEM STOP MERGES t_json;
INSERT INTO t_json SELECT number, '{"k1": 1, "k2": 2}' FROM numbers(1000000);
INSERT INTO t_json VALUES (1000001, '{"foo": 1}');
SELECT toTypeName(obj) FROM t_json LIMIT 1;
SELECT count() FROM t_json WHERE obj.foo != 0;
DROP TABLE IF EXISTS t_json;

View File

@ -0,0 +1,8 @@
{"id":"1","obj":{"k1":1,"k2":{"k3":"2","k4":[{"k5":3,"k6":0},{"k5":4,"k6":0}]},"some":0},"s":"foo"}
{"id":"2","obj":{"k1":0,"k2":{"k3":"str","k4":[{"k5":0,"k6":55}]},"some":42},"s":"bar"}
Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8)
{"id":"1","obj":"aaa","s":"foo"}
{"id":"2","obj":"bbb","s":"bar"}
{"map":{"k1":1,"k2":2},"obj":{"k1":1,"k2.k3":2},"map_type":"Map(String, Nullable(Float64))","obj_type":"Object('json')"}
{"obj":{"k1":1,"k2":2},"map":{"k1":"1","k2":"2"}}
Tuple(k1 Float64, k2 Float64)

View File

@ -0,0 +1,52 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (id UInt64, obj JSON, s String) \
ENGINE = MergeTree ORDER BY id" --allow_experimental_object_type 1
user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/
rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/*
filename="${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json"
echo '{"id": 1, "obj": {"k1": 1, "k2": {"k3": 2, "k4": [{"k5": 3}, {"k5": 4}]}}, "s": "foo"}' > $filename
echo '{"id": 2, "obj": {"k2": {"k3": "str", "k4": [{"k6": 55}]}, "some": 42}, "s": "bar"}' >> $filename
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')"
${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (id UInt64, obj String, s String) ENGINE = MergeTree ORDER BY id"
echo '{"obj": "aaa", "id": 1, "s": "foo"}' > $filename
echo '{"id": 2, "obj": "bbb", "s": "bar"}' >> $filename
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')"
${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference"
echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": {"k3": 2}}}' > $filename
${CLICKHOUSE_CLIENT} -q "SELECT map, obj, toTypeName(map) AS map_type, toTypeName(obj) AS obj_type \
FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow') FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (obj JSON, map Map(String, UInt64)) \
ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": 2}}' > $filename
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')"
${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference"

View File

@ -0,0 +1,6 @@
10
0
1
2
3
4

View File

@ -0,0 +1,22 @@
DROP TABLE IF EXISTS t_max_rows_to_read;
CREATE TABLE t_max_rows_to_read (a UInt64)
ENGINE = MergeTree ORDER BY a
SETTINGS index_granularity = 4;
INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100);
SET max_threads = 1;
SELECT a FROM t_max_rows_to_read WHERE a = 10 SETTINGS max_rows_to_read = 4;
SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12;
-- This should work, but actually it doesn't. Need to investigate.
-- SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 20;
SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 20 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 }
SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 }
SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 FORMAT Null SETTINGS max_rows_to_read = 4; -- { serverError 158 }
DROP TABLE t_max_rows_to_read;

View File

@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02246"
${CLICKHOUSE_CLIENT} -q "CREATE USER u02246"
${CLICKHOUSE_CLIENT} -q "GRANT INSERT ON async_inserts_02246 TO r02246"
${CLICKHOUSE_CLIENT} -q "GRANT r02246 to u02246"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02246 FOR INTERVAL 1 HOUR MAX QUERY INSERTS = 2 TO r02246"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02246 FOR INTERVAL 100 YEAR MAX QUERY INSERTS = 2 TO r02246"
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (1, 'a')"
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (2, 'b')"

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
dir=${CLICKHOUSE_TEST_UNIQUE_NAME}
[[ -d $dir ]] && rm -r $dir
mkdir $dir
$CLICKHOUSE_LOCAL --multiline --multiquery --path $dir -q """
DROP DATABASE IF EXISTS test;
CREATE DATABASE IF NOT EXISTS test;
USE test;
CREATE TABLE test (id Int32) ENGINE=MergeTree() ORDER BY id;
DROP DATABASE test;
"""
$CLICKHOUSE_LOCAL --multiline --multiquery -q """
DROP DATABASE IF EXISTS test;
CREATE DATABASE IF NOT EXISTS test;
USE test;
CREATE TABLE test (id Int32) ENGINE=MergeTree() ORDER BY id;
DROP DATABASE test;
"""

View File

@ -0,0 +1,4 @@
([1,2],['a','b'],3,'c',4) Tuple(`t1.a` Array(UInt32), `t1.s` Array(String), b UInt32, `t2.k` String, `t2.v` UInt32)
Tuple(id Int8, obj Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8), s String) Tuple(id Int8, `obj.k1` Int8, `obj.k2.k3` String, `obj.k2.k4.k5` Array(Int8), `obj.k2.k4.k6` Array(Int8), `obj.some` Int8, s String)
1 1 2 [3,4] [0,0] 0 foo
2 0 str [0] [55] 42 bar

View File

@ -0,0 +1,24 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS t_flatten_tuple;
DROP TABLE IF EXISTS t_flatten_object;
SET flatten_nested = 0;
CREATE TABLE t_flatten_tuple(t Tuple(t1 Nested(a UInt32, s String), b UInt32, t2 Tuple(k String, v UInt32))) ENGINE = Memory;
INSERT INTO t_flatten_tuple VALUES (([(1, 'a'), (2, 'b')], 3, ('c', 4)));
SELECT flattenTuple(t) AS ft, toTypeName(ft) FROM t_flatten_tuple;
SET allow_experimental_object_type = 1;
CREATE TABLE t_flatten_object(data JSON) ENGINE = Memory;
INSERT INTO t_flatten_object VALUES ('{"id": 1, "obj": {"k1": 1, "k2": {"k3": 2, "k4": [{"k5": 3}, {"k5": 4}]}}, "s": "foo"}');
INSERT INTO t_flatten_object VALUES ('{"id": 2, "obj": {"k2": {"k3": "str", "k4": [{"k6": 55}]}, "some": 42}, "s": "bar"}');
SELECT toTypeName(data), toTypeName(flattenTuple(data)) FROM t_flatten_object LIMIT 1;
SELECT untuple(flattenTuple(data)) FROM t_flatten_object ORDER BY data.id;
DROP TABLE IF EXISTS t_flatten_tuple;
DROP TABLE IF EXISTS t_flatten_object;