mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge remote-tracking branch 'ck/master' into use_minmax_index
This commit is contained in:
commit
fcfabf4a0c
@ -1,9 +1,12 @@
|
||||
if (APPLE OR NOT ARCH_AMD64 OR SANITIZE STREQUAL "undefined")
|
||||
set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF)
|
||||
# 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)
|
||||
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)
|
||||
|
@ -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
|
||||
|
@ -8,7 +8,7 @@ toc_title: "版本折叠MergeTree"
|
||||
这个引擎:
|
||||
|
||||
- 允许快速写入不断变化的对象状态。
|
||||
- 删除后台中的旧对象状态。 这显着降低了存储体积。
|
||||
- 删除后台中的旧对象状态。 这显著降低了存储体积。
|
||||
|
||||
请参阅部分 [崩溃](#table_engines_versionedcollapsingmergetree) 有关详细信息。
|
||||
|
||||
|
@ -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));
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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()
|
||||
|
@ -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")
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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)
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
if (isObject(type))
|
||||
{
|
||||
is_object = true;
|
||||
break;
|
||||
}
|
||||
|
||||
value_type = type;
|
||||
if (!value_type)
|
||||
{
|
||||
value_type = type;
|
||||
}
|
||||
else if (!value_type->equals(*type))
|
||||
{
|
||||
is_object = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!value_type)
|
||||
return nullptr;
|
||||
if (is_object)
|
||||
return std::make_shared<DataTypeObject>("json", false);
|
||||
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
|
||||
if (value_type)
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"};
|
||||
|
@ -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);
|
||||
|
@ -53,6 +53,7 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -3140,52 +3141,138 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
WrapperType createTupleToObjectWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const
|
||||
{
|
||||
if (!from_tuple.haveExplicitNames())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"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_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_tuple.getName());
|
||||
|
||||
type = recursiveRemoveLowCardinality(type);
|
||||
}
|
||||
|
||||
return [element_wrappers = getElementWrappers(from_types, to_types),
|
||||
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();
|
||||
auto flattened_column = flattenTuple(arguments.front().column);
|
||||
const auto & column_tuple = assert_cast<const ColumnTuple &>(*flattened_column);
|
||||
|
||||
if (tuple_size != column_tuple.getColumns().size())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Expected tuple with {} subcolumn, but got {} subcolumns",
|
||||
tuple_size, column_tuple.getColumns().size());
|
||||
|
||||
auto res = ColumnObject::create(has_nullable_subcolumns);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }};
|
||||
auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count);
|
||||
res->addSubcolumn(paths[i], converted_column->assumeMutable());
|
||||
}
|
||||
|
||||
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()))
|
||||
{
|
||||
if (!from_tuple->haveExplicitNames())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_type->getName());
|
||||
|
||||
PathsInData paths;
|
||||
DataTypes from_types;
|
||||
|
||||
std::tie(paths, from_types) = flattenTuple(from_type);
|
||||
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());
|
||||
|
||||
type = recursiveRemoveLowCardinality(type);
|
||||
}
|
||||
|
||||
return [element_wrappers = getElementWrappers(from_types, to_types),
|
||||
has_nullable_subcolumns = to_type->hasNullableSubcolumns(), 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();
|
||||
auto flattened_column = flattenTuple(arguments.front().column);
|
||||
const auto & column_tuple = assert_cast<const ColumnTuple &>(*flattened_column);
|
||||
|
||||
if (tuple_size != column_tuple.getColumns().size())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Expected tuple with {} subcolumn, but got {} subcolumns",
|
||||
tuple_size, column_tuple.getColumns().size());
|
||||
|
||||
auto res = ColumnObject::create(has_nullable_subcolumns);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }};
|
||||
auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count);
|
||||
res->addSubcolumn(paths[i], converted_column->assumeMutable());
|
||||
}
|
||||
|
||||
return res;
|
||||
};
|
||||
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()))
|
||||
{
|
||||
@ -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>
|
||||
|
68
src/Functions/flattenTuple.cpp
Normal file
68
src/Functions/flattenTuple.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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) {}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
|
@ -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.
|
||||
|
@ -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)};
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
11
src/Storages/MergeTree/IPartMetadataManager.cpp
Normal file
11
src/Storages/MergeTree/IPartMetadataManager.cpp
Normal 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())
|
||||
{
|
||||
}
|
||||
}
|
59
src/Storages/MergeTree/IPartMetadataManager.h
Normal file
59
src/Storages/MergeTree/IPartMetadataManager.h
Normal 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>;
|
||||
}
|
@ -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;
|
||||
|
||||
|
@ -442,4 +442,6 @@ private:
|
||||
bool strict;
|
||||
};
|
||||
|
||||
String extractFixedPrefixFromLikePattern(const String & like_pattern);
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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
|
||||
|
107
src/Storages/MergeTree/MergeTreeMetadataCache.cpp
Normal file
107
src/Storages/MergeTree/MergeTreeMetadataCache.cpp
Normal 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
|
45
src/Storages/MergeTree/MergeTreeMetadataCache.h
Normal file
45
src/Storages/MergeTree/MergeTreeMetadataCache.h
Normal 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
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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) \
|
||||
|
33
src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp
Normal file
33
src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp
Normal 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);
|
||||
}
|
||||
|
||||
|
||||
}
|
29
src/Storages/MergeTree/PartMetadataManagerOrdinary.h
Normal file
29
src/Storages/MergeTree/PartMetadataManagerOrdinary.h
Normal 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 {}; }
|
||||
};
|
||||
|
||||
|
||||
}
|
273
src/Storages/MergeTree/PartMetadataManagerWithCache.cpp
Normal file
273
src/Storages/MergeTree/PartMetadataManagerWithCache.cpp
Normal 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
|
56
src/Storages/MergeTree/PartMetadataManagerWithCache.h
Normal file
56
src/Storages/MergeTree/PartMetadataManagerWithCache.h
Normal 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
|
@ -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, ""};
|
||||
}
|
||||
|
||||
|
@ -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
|
@ -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)
|
||||
|
@ -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)
|
||||
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 column = columns.tryGetColumn(options, name);
|
||||
if (column && !isObject(column->type))
|
||||
{
|
||||
res.emplace_back(std::move(*column));
|
||||
continue;
|
||||
}
|
||||
|
||||
if (options.with_extended_objects)
|
||||
{
|
||||
auto object_column = object_columns.tryGetColumn(options, name);
|
||||
if (object_column)
|
||||
{
|
||||
res.emplace_back(std::move(*object_column));
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (options.with_virtuals)
|
||||
{
|
||||
auto it = virtual_columns.find(name);
|
||||
if (it != virtual_columns.end())
|
||||
{
|
||||
res.emplace_back(name, it->second);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", name);
|
||||
auto object_column = object_columns.tryGetColumn(options, column_name);
|
||||
if (object_column)
|
||||
return object_column;
|
||||
}
|
||||
|
||||
return res;
|
||||
if (options.with_virtuals)
|
||||
{
|
||||
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
|
||||
{
|
||||
auto column = tryGetColumn(options, column_name);
|
||||
if (!column)
|
||||
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", column_name);
|
||||
|
||||
return *column;
|
||||
}
|
||||
|
||||
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
143
src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp
Normal file
143
src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp
Normal 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
|
32
src/Storages/System/StorageSystemMergeTreeMetadataCache.h
Normal file
32
src/Storages/System/StorageSystemMergeTreeMetadataCache.h
Normal 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
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -10,13 +10,13 @@ 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")
|
||||
self.auth = {
|
||||
"X-ClickHouse-User": get_parameter_from_ssm(
|
||||
"clickhouse-test-stat-login2"
|
||||
),
|
||||
"X-ClickHouse-Key": "",
|
||||
}
|
||||
url = get_parameter_from_ssm("clickhouse-test-stat-url")
|
||||
|
||||
self.url = url
|
||||
self.auth = {
|
||||
"X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"),
|
||||
"X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password")
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
def _insert_json_str_info_impl(url, auth, db, table, json_str):
|
||||
@ -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))
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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__":
|
||||
|
@ -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__":
|
||||
|
@ -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)
|
||||
|
@ -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":
|
||||
|
@ -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:
|
||||
|
@ -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)
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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")
|
||||
|
@ -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")]
|
||||
|
@ -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>
|
||||
|
@ -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(
|
||||
|
@ -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
|
||||
|
||||
|
@ -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[@]}"
|
||||
|
@ -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
|
101
tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh
Executable file
101
tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh
Executable 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
|
@ -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
|
41
tests/queries/0_stateless/01825_type_json_from_map.sql
Normal file
41
tests/queries/0_stateless/01825_type_json_from_map.sql
Normal 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;
|
@ -0,0 +1,2 @@
|
||||
Tuple(foo Int8, k1 Int8, k2 Int8)
|
||||
1
|
19
tests/queries/0_stateless/01825_type_json_missed_values.sql
Normal file
19
tests/queries/0_stateless/01825_type_json_missed_values.sql
Normal 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;
|
@ -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)
|
52
tests/queries/0_stateless/01825_type_json_schema_inference.sh
Executable file
52
tests/queries/0_stateless/01825_type_json_schema_inference.sh
Executable 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"
|
@ -0,0 +1,6 @@
|
||||
10
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
@ -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;
|
@ -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')"
|
||||
|
24
tests/queries/0_stateless/02246_clickhouse_local_drop_database.sh
Executable file
24
tests/queries/0_stateless/02246_clickhouse_local_drop_database.sh
Executable 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;
|
||||
"""
|
4
tests/queries/0_stateless/02246_flatten_tuple.reference
Normal file
4
tests/queries/0_stateless/02246_flatten_tuple.reference
Normal 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
|
24
tests/queries/0_stateless/02246_flatten_tuple.sql
Normal file
24
tests/queries/0_stateless/02246_flatten_tuple.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user