Merge branch 'master' of github.com:ClickHouse/ClickHouse into enable-coverage-for-debug-build

This commit is contained in:
Alexey Milovidov 2024-01-30 03:53:11 +01:00
commit eb54432cbe
54 changed files with 427 additions and 198 deletions

View File

@ -904,13 +904,6 @@ public:
/// Returns a pointer to the Logger with the given name if it
/// exists, or a null pointer otherwise.
static bool destroy(const std::string & name);
/// Destroys the logger with the specified name. Does nothing
/// if the logger is not found.
///
/// After a logger has been destroyed, all references to it
/// become invalid.
static void shutdown();
/// Shuts down the logging framework and releases all
/// Loggers.
@ -940,8 +933,6 @@ public:
static const std::string ROOT; /// The name of the root logger ("").
protected:
typedef std::map<std::string, Logger *> LoggerMap;
Logger(const std::string & name, Channel * pChannel, int level);
~Logger();
@ -962,8 +953,6 @@ private:
std::string _name;
Channel * _pChannel;
std::atomic_int _level;
static LoggerMap * _pLoggerMap;
};

View File

@ -20,6 +20,7 @@
#include "Poco/NumberParser.h"
#include "Poco/String.h"
#include <cassert>
#include <mutex>
namespace
@ -37,12 +38,20 @@ std::mutex & getLoggerMutex()
return *logger_mutex;
}
struct LoggerEntry
{
Poco::Logger * logger;
bool owned_by_shared_ptr = false;
};
using LoggerMap = std::unordered_map<std::string, LoggerEntry>;
LoggerMap * _pLoggerMap = nullptr;
}
namespace Poco {
Logger::LoggerMap* Logger::_pLoggerMap = 0;
const std::string Logger::ROOT;
@ -134,12 +143,12 @@ void Logger::setLevel(const std::string& name, int level)
if (_pLoggerMap)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setLevel(level);
it.second.logger->setLevel(level);
}
}
}
@ -153,12 +162,12 @@ void Logger::setChannel(const std::string& name, Channel* pChannel)
if (_pLoggerMap)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setChannel(pChannel);
it.second.logger->setChannel(pChannel);
}
}
}
@ -172,12 +181,12 @@ void Logger::setProperty(const std::string& loggerName, const std::string& prope
if (_pLoggerMap)
{
std::string::size_type len = loggerName.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, loggerName) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, loggerName) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setProperty(propertyName, value);
it.second.logger->setProperty(propertyName, value);
}
}
}
@ -304,35 +313,84 @@ struct LoggerDeleter
{
void operator()(Poco::Logger * logger)
{
if (Logger::destroy(logger->name()))
return;
std::lock_guard<std::mutex> lock(getLoggerMutex());
logger->release();
/// If logger infrastructure is destroyed just decrement logger reference count
if (!_pLoggerMap)
{
logger->release();
return;
}
auto it = _pLoggerMap->find(logger->name());
assert(it != _pLoggerMap->end());
/** If reference count is 1, this means this shared pointer owns logger
* and need destroy it.
*/
size_t reference_count_before_release = logger->release();
if (reference_count_before_release == 1)
{
assert(it->second.owned_by_shared_ptr);
_pLoggerMap->erase(it);
}
}
};
inline LoggerPtr makeLoggerPtr(Logger & logger)
{
logger.duplicate();
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
}
}
Logger& Logger::get(const std::string& name)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(name);
Logger & logger = unsafeGet(name);
/** If there are already shared pointer created for this logger
* we need to increment Logger reference count and now logger
* is owned by logger infrastructure.
*/
auto it = _pLoggerMap->find(name);
if (it->second.owned_by_shared_ptr)
{
it->second.logger->duplicate();
it->second.owned_by_shared_ptr = false;
}
return logger;
}
LoggerPtr Logger::getShared(const std::string & name)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name);
return makeLoggerPtr(unsafeGet(name));
Logger & logger = unsafeGet(name);
/** If logger already exists, then this shared pointer does not own it.
* If logger does not exists, logger infrastructure could be already destroyed
* or logger was created.
*/
if (logger_exists)
{
logger.duplicate();
}
else if (_pLoggerMap)
{
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
}
return makeLoggerPtr(logger);
}
Logger& Logger::unsafeGet(const std::string& name)
{
Logger* pLogger = find(name);
@ -364,7 +422,10 @@ LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
return makeLoggerPtr(unsafeCreate(name, pChannel, level));
Logger & logger = unsafeCreate(name, pChannel, level);
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
return makeLoggerPtr(logger);
}
Logger& Logger::root()
@ -389,10 +450,14 @@ void Logger::shutdown()
if (_pLoggerMap)
{
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
it->second->release();
if (it.second.owned_by_shared_ptr)
continue;
it.second.logger->release();
}
delete _pLoggerMap;
_pLoggerMap = 0;
}
@ -405,32 +470,12 @@ Logger* Logger::find(const std::string& name)
{
LoggerMap::iterator it = _pLoggerMap->find(name);
if (it != _pLoggerMap->end())
return it->second;
return it->second.logger;
}
return 0;
}
bool Logger::destroy(const std::string& name)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
LoggerMap::iterator it = _pLoggerMap->find(name);
if (it != _pLoggerMap->end())
{
if (it->second->release() == 1)
_pLoggerMap->erase(it);
return true;
}
}
return false;
}
void Logger::names(std::vector<std::string>& names)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
@ -538,7 +583,8 @@ void Logger::add(Logger* pLogger)
{
if (!_pLoggerMap)
_pLoggerMap = new LoggerMap;
_pLoggerMap->insert(LoggerMap::value_type(pLogger->name(), pLogger));
_pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/});
}

View File

@ -79,7 +79,10 @@ if (SANITIZE_COVERAGE)
# But the actual coverage will be enabled on per-library basis: for ClickHouse code, but not for 3rd-party.
set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard,pc-table")
endif()
set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table")
set (WITHOUT_COVERAGE_FLAGS_LIST -fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table)
set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table")
set (WITHOUT_COVERAGE_FLAGS_LIST -fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table)
else()
set (WITHOUT_COVERAGE_FLAGS "")
set (WITHOUT_COVERAGE_FLAGS_LIST "")
endif()

2
contrib/simdjson vendored

@ -1 +1 @@
Subproject commit 1075e8609c4afa253162d441437af929c29e31bb
Subproject commit 6060be2fdf62edf4a8f51a8b0883d57d09397b30

View File

@ -287,7 +287,7 @@ Number of threads in the HashedDictionary thread pool running a task.
### IOPrefetchThreads
Number of threads in the IO prefertch thread pool.
Number of threads in the IO prefetch thread pool.
### IOPrefetchThreadsActive

View File

@ -77,8 +77,8 @@ The number of data points in `series` should be at least twice the value of `per
**Returned value**
- An array of three arrays where the first array include seasonal components, the second array - trend,
and the third array - residue component.
- An array of four arrays where the first array include seasonal components, the second array - trend,
the third array - residue component, and the fourth array - baseline(seasonal + trend) component.
Type: [Array](../../sql-reference/data-types/array.md).
@ -107,6 +107,10 @@ Result:
[
0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0,
0
],
[
10.1, 20.449999, 40.340004, 10.100001, 20.45, 40.34, 10.100001, 20.45, 40.34, 10.1, 20.45, 40.34,
10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.100002, 20.45, 40.34
]] │
└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```

View File

@ -294,12 +294,14 @@ if (TARGET ch_contrib::gwp_asan)
endif()
# Otherwise it will slow down stack traces printing too much.
set_source_files_properties(
Common/Elf.cpp
Common/Dwarf.cpp
Common/SymbolIndex.cpp
Common/ThreadFuzzer.cpp
PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE_FLAGS}")
if ("${CMAKE_BUILD_TYPE_UC}" STREQUAL "DEBUG")
set_source_files_properties(
Common/Elf.cpp
Common/Dwarf.cpp
Common/SymbolIndex.cpp
Common/ThreadFuzzer.cpp
PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE_FLAGS}")
endif()
target_link_libraries (clickhouse_common_io
PRIVATE

51
src/Common/AtomicLogger.h Normal file
View File

@ -0,0 +1,51 @@
#pragma once
#include <memory>
#include <Common/Logger.h>
#include <Common/SharedMutex.h>
#include <Common/SharedLockGuard.h>
/** AtomicLogger allows to atomically change logger.
* Standard library does not have atomic_shared_ptr, and we do not use std::atomic* operations,
* because standard library implementation uses fixed table of mutexes, and it is better to avoid contention here.
*/
class AtomicLogger
{
public:
explicit AtomicLogger(LoggerPtr logger_)
: logger(std::move(logger_))
{}
explicit AtomicLogger(const std::string & log_name)
: AtomicLogger(::getLogger(log_name))
{}
void store(LoggerPtr new_logger)
{
std::lock_guard lock(log_mutex);
logger = std::move(new_logger);
}
void store(const std::string & new_log_name)
{
auto new_logger = ::getLogger(new_log_name);
store(std::move(new_logger));
}
LoggerPtr load() const
{
DB::SharedLockGuard lock(log_mutex);
return logger;
}
String loadName() const
{
DB::SharedLockGuard lock(log_mutex);
return logger->name();
}
private:
mutable DB::SharedMutex log_mutex;
LoggerPtr logger;
};

View File

@ -87,7 +87,7 @@
M(MarksLoaderThreads, "Number of threads in thread pool for loading marks.") \
M(MarksLoaderThreadsActive, "Number of threads in the thread pool for loading marks running a task.") \
M(MarksLoaderThreadsScheduled, "Number of queued or active jobs in the thread pool for loading marks.") \
M(IOPrefetchThreads, "Number of threads in the IO prefertch thread pool.") \
M(IOPrefetchThreads, "Number of threads in the IO prefetch thread pool.") \
M(IOPrefetchThreadsActive, "Number of threads in the IO prefetch thread pool running a task.") \
M(IOPrefetchThreadsScheduled, "Number of queued or active jobs in the IO prefetch thread pool.") \
M(IOWriterThreads, "Number of threads in the IO writer thread pool.") \

View File

@ -257,6 +257,11 @@ void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_messa
tryLogCurrentException(logger.get(), start_of_message);
}
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message)
{
tryLogCurrentException(logger.load(), start_of_message);
}
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg)
{
path = std::filesystem::absolute(path);
@ -529,6 +534,11 @@ void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string &
}
}
void tryLogException(std::exception_ptr e, const AtomicLogger & logger, const std::string & start_of_message)
{
tryLogException(e, logger.load(), start_of_message);
}
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text;

View File

@ -11,6 +11,7 @@
#include <base/scope_guard.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/Logger.h>
#include <Common/AtomicLogger.h>
#include <Common/StackTrace.h>
#include <fmt/format.h>
@ -245,6 +246,7 @@ using Exceptions = std::vector<std::exception_ptr>;
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "");
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "");
void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = "");
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = "");
/** Prints current exception in canonical format.
@ -290,6 +292,7 @@ struct ExecutionStatus
/// TODO: Logger leak constexpr overload
void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = "");
void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string & start_of_message = "");
void tryLogException(std::exception_ptr e, const AtomicLogger & logger, const std::string & start_of_message = "");
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);

View File

@ -71,6 +71,28 @@ struct StringHashTableHash
res = _mm_crc32_u64(res, key.c);
return res;
}
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
size_t ALWAYS_INLINE operator()(StringKey8 key) const
{
size_t res = -1ULL;
res = __crc32cd(static_cast<UInt32>(res), key);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey16 key) const
{
size_t res = -1ULL;
res = __crc32cd(static_cast<UInt32>(res), key.items[0]);
res = __crc32cd(static_cast<UInt32>(res), key.items[1]);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey24 key) const
{
size_t res = -1ULL;
res = __crc32cd(static_cast<UInt32>(res), key.a);
res = __crc32cd(static_cast<UInt32>(res), key.b);
res = __crc32cd(static_cast<UInt32>(res), key.c);
return res;
}
#elif defined(__s390x__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
size_t ALWAYS_INLINE operator()(StringKey8 key) const
{

View File

@ -43,7 +43,6 @@ LoggerRawPtr getRawLogger(const std::string & name);
LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION);
/** Returns true, if currently Logger with specified name is created.
* Otherwise, returns false.
*/

View File

@ -1,6 +1,7 @@
#if defined(__ELF__) && !defined(OS_FREEBSD)
#include <Common/SymbolIndex.h>
#include <Common/MemorySanitizer.h>
#include <algorithm>
#include <optional>
@ -55,21 +56,6 @@ Otherwise you will get only exported symbols from program headers.
*/
#if defined(__clang__)
# pragma clang diagnostic ignored "-Wreserved-id-macro"
# pragma clang diagnostic ignored "-Wunused-macros"
#endif
#define __msan_unpoison_string(X) // NOLINT
#define __msan_unpoison(X, Y) // NOLINT
#if defined(ch_has_feature)
# if ch_has_feature(memory_sanitizer)
# undef __msan_unpoison_string
# undef __msan_unpoison
# include <sanitizer/msan_interface.h>
# endif
#endif
namespace DB
{

View File

@ -9,6 +9,7 @@
#include <Common/ProfileEvents.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/Logger.h>
#include <Common/AtomicLogger.h>
namespace Poco { class Logger; }
@ -20,7 +21,8 @@ using LogSeriesLimiterPtr = std::shared_ptr<LogSeriesLimiter>;
namespace
{
[[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const LoggerPtr & logger) { return logger.get(); }
[[maybe_unused]] LoggerPtr getLoggerHelper(const LoggerPtr & logger) { return logger; }
[[maybe_unused]] LoggerPtr getLoggerHelper(const AtomicLogger & logger) { return logger.load(); }
[[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; }
[[maybe_unused]] std::unique_ptr<LogToStrImpl> getLoggerHelper(std::unique_ptr<LogToStrImpl> && logger) { return logger; }
[[maybe_unused]] std::unique_ptr<LogFrequencyLimiterIml> getLoggerHelper(std::unique_ptr<LogFrequencyLimiterIml> && logger) { return logger; }

View File

@ -32,7 +32,6 @@ TEST(Logger, TestLog)
LOG_TEST(log, "Hello World");
EXPECT_EQ(oss.str(), "Hello World\n");
Poco::Logger::destroy("TestLogger");
}
{ /// Test logs invisible for other levels
@ -45,8 +44,6 @@ TEST(Logger, TestLog)
LOG_TEST(log, "Hello World");
EXPECT_EQ(oss.str(), "");
Poco::Logger::destroy(std::string{level} + "_Logger");
}
}

View File

@ -6,14 +6,15 @@
#include <Compression/CompressionFactory.h>
#include <Compression/CompressionInfo.h>
#include <Poco/Logger.h>
#include <Common/randomSeed.h>
#include <Common/logger_useful.h>
#include "libaccel_config.h"
#include <Common/MemorySanitizer.h>
#include <Common/logger_useful.h>
#include <Common/randomSeed.h>
#include <base/scope_guard.h>
#include <base/getPageSize.h>
#include <immintrin.h>
#include "libaccel_config.h"
#include <immintrin.h>
namespace DB
{
@ -416,9 +417,7 @@ UInt32 CompressionCodecDeflateQpl::doCompressData(const char * source, UInt32 so
{
/// QPL library is using AVX-512 with some shuffle operations.
/// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.
#if defined(MEMORY_SANITIZER)
__msan_unpoison(dest, getMaxCompressedDataSize(source_size));
#endif
Int32 res = HardwareCodecDeflateQpl::RET_ERROR;
if (DeflateQplJobHWPool::instance().isJobPoolReady())
res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size));
@ -439,9 +438,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so
{
/// QPL library is using AVX-512 with some shuffle operations.
/// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.
#if defined(MEMORY_SANITIZER)
__msan_unpoison(dest, uncompressed_size);
#endif
/// Device IOTLB miss has big perf. impact for IAA accelerators.
/// To avoid page fault, we need touch buffers related to accelerator in advance.
touchBufferWithZeroFilling(dest, uncompressed_size);

View File

@ -102,7 +102,9 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"},
{"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"},
{"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"},
{"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}}},
{"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"},
{"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"},
{"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}}},
{"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."},
{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"},
{"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"},

View File

@ -46,7 +46,7 @@ IVolume::IVolume(
}
if (disks.empty())
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Volume must contain at least one disk");
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Volume {} must contain at least one disk", name);
}
std::optional<UInt64> IVolume::getMaxUnreservedFreeSpace() const

View File

@ -3,6 +3,7 @@
#include <Functions/FunctionsHashing.h>
#include <Common/HashTable/ClearableHashMap.h>
#include <Common/HashTable/Hash.h>
#include <Common/MemorySanitizer.h>
#include <Common/UTF8Helpers.h>
#include <Core/Defines.h>
@ -108,10 +109,8 @@ struct NgramDistanceImpl
if constexpr (case_insensitive)
{
#if defined(MEMORY_SANITIZER)
/// Due to PODArray padding accessing more elements should be OK
__msan_unpoison(code_points + (N - 1), padding_offset * sizeof(CodePoint));
#endif
/// We really need template lambdas with C++20 to do it inline
unrollLowering<N - 1>(code_points, std::make_index_sequence<padding_offset>());
}

View File

@ -128,6 +128,10 @@ public:
res_data.insert(residue.begin(), residue.end());
res_col_offsets_data.push_back(res_data.size());
// Create Baseline = seasonal + trend
std::transform(seasonal.begin(), seasonal.end(), trend.begin(), std::back_inserter(res_data), std::plus<>());
res_col_offsets_data.push_back(res_data.size());
root_offsets_data.push_back(res_col_offsets->size());
prev_src_offset = curr_offset;
@ -201,7 +205,7 @@ The number of data points in `series` should be at least twice the value of `per
**Returned value**
- An array of three arrays where the first array include seasonal components, the second array - trend, and the third array - residue component.
- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline(seasonal + trend) component.
Type: [Array](../../sql-reference/data-types/array.md).
@ -230,6 +234,10 @@ Result:
[
0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0,
0
],
[
10.1, 20.449999, 40.340004, 10.100001, 20.45, 40.34, 10.100001, 20.45, 40.34, 10.1, 20.45, 40.34,
10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.100002, 20.45, 40.34
]]
```)",

View File

@ -1435,8 +1435,14 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
interpreter.execute();
}
else
throw Exception(storage_already_exists_error_code,
"{} {}.{} already exists", storage_name, backQuoteIfNeed(create.getDatabase()), backQuoteIfNeed(create.getTable()));
{
if (database->getTable(create.getTable(), getContext())->isDictionary())
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS,
"Dictionary {}.{} already exists", backQuoteIfNeed(create.getDatabase()), backQuoteIfNeed(create.getTable()));
else
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS,
"Table {}.{} already exists", backQuoteIfNeed(create.getDatabase()), backQuoteIfNeed(create.getTable()));
}
}
else if (!create.attach)
{

View File

@ -17,9 +17,14 @@ namespace ErrorCodes
}
ColumnGathererStream::ColumnGathererStream(
size_t num_inputs, ReadBuffer & row_sources_buf_, size_t block_preferred_size_)
: sources(num_inputs), row_sources_buf(row_sources_buf_)
, block_preferred_size(block_preferred_size_)
size_t num_inputs,
ReadBuffer & row_sources_buf_,
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_)
: sources(num_inputs)
, row_sources_buf(row_sources_buf_)
, block_preferred_size_rows(block_preferred_size_rows_)
, block_preferred_size_bytes(block_preferred_size_bytes_)
{
if (num_inputs == 0)
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "There are no streams to gather");
@ -124,10 +129,11 @@ ColumnGathererTransform::ColumnGathererTransform(
const Block & header,
size_t num_inputs,
ReadBuffer & row_sources_buf_,
size_t block_preferred_size_)
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_)
: IMergingTransform<ColumnGathererStream>(
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
num_inputs, row_sources_buf_, block_preferred_size_)
num_inputs, row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_)
, log(getLogger("ColumnGathererStream"))
{
if (header.columns() != 1)

View File

@ -5,7 +5,6 @@
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
#include <Processors/Merges/IMergingTransform.h>
namespace Poco { class Logger; }
@ -57,7 +56,11 @@ using MergedRowSources = PODArray<RowSourcePart>;
class ColumnGathererStream final : public IMergingAlgorithm
{
public:
ColumnGathererStream(size_t num_inputs, ReadBuffer & row_sources_buf_, size_t block_preferred_size_ = DEFAULT_BLOCK_SIZE);
ColumnGathererStream(
size_t num_inputs,
ReadBuffer & row_sources_buf_,
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_);
const char * getName() const override { return "ColumnGathererStream"; }
void initialize(Inputs inputs) override;
@ -92,13 +95,12 @@ private:
std::vector<Source> sources;
ReadBuffer & row_sources_buf;
const size_t block_preferred_size;
const size_t block_preferred_size_rows;
const size_t block_preferred_size_bytes;
Source * source_to_fully_copy = nullptr;
ssize_t next_required_source = -1;
size_t cur_block_preferred_size = 0;
UInt64 merged_rows = 0;
UInt64 merged_bytes = 0;
};
@ -110,7 +112,8 @@ public:
const Block & header,
size_t num_inputs,
ReadBuffer & row_sources_buf_,
size_t block_preferred_size_ = DEFAULT_BLOCK_SIZE);
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_);
String getName() const override { return "ColumnGathererTransform"; }
@ -134,14 +137,17 @@ void ColumnGathererStream::gather(Column & column_res)
if (next_required_source == -1)
{
/// Start new column.
cur_block_preferred_size = std::min(static_cast<size_t>(row_sources_end - row_source_pos), block_preferred_size);
column_res.reserve(cur_block_preferred_size);
/// Actually reserve works only for fixed size columns.
/// So it's safe to ignore preferred size in bytes and call reserve for number of rows.
size_t size_to_reserve = std::min(static_cast<size_t>(row_sources_end - row_source_pos), block_preferred_size_rows);
column_res.reserve(size_to_reserve);
}
size_t cur_size = column_res.size();
next_required_source = -1;
while (row_source_pos < row_sources_end && cur_size < cur_block_preferred_size)
while (row_source_pos < row_sources_end
&& column_res.size() < block_preferred_size_rows
&& column_res.allocatedBytes() < block_preferred_size_bytes)
{
RowSourcePart row_source = *row_source_pos;
size_t source_num = row_source.getSourceNum();
@ -159,6 +165,7 @@ void ColumnGathererStream::gather(Column & column_res)
/// Consecutive optimization. TODO: precompute lengths
size_t len = 1;
size_t max_len = std::min(static_cast<size_t>(row_sources_end - row_source_pos), source.size - source.pos); // interval should be in the same block
while (len < max_len && row_source_pos->data == row_source.data)
{
++len;
@ -181,8 +188,6 @@ void ColumnGathererStream::gather(Column & column_res)
column_res.insertFrom(*source.column, source.pos);
else
column_res.insertRangeFrom(*source.column, source.pos, len);
cur_size += len;
}
source.pos += len;

View File

@ -1663,7 +1663,7 @@ try
metadata_manager->deleteAll(true);
metadata_manager->assertAllDeleted(true);
getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log, remove_new_dir_if_exists, fsync_dir);
getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log.load(), remove_new_dir_if_exists, fsync_dir);
metadata_manager->updateAll(true);
auto new_projection_root_path = to.string();
@ -1758,7 +1758,7 @@ void IMergeTreeDataPart::remove()
}
bool is_temporary_part = is_temp || state == MergeTreeDataPartState::Temporary;
getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log);
getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log.load());
}
std::optional<String> IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached, bool broken) const
@ -1775,7 +1775,7 @@ std::optional<String> IMergeTreeDataPart::getRelativePathForPrefix(const String
if (detached && parent_part)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection");
return getDataPartStorage().getRelativePathForPrefix(storage.log, prefix, detached, broken);
return getDataPartStorage().getRelativePathForPrefix(storage.log.load(), prefix, detached, broken);
}
std::optional<String> IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix, bool broken) const
@ -1841,7 +1841,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name);
String path_to_clone = fs::path(storage.relative_data_path) / directory_name / "";
return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log, cancellation_hook);
return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log.load(), cancellation_hook);
}
UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const

View File

@ -588,7 +588,15 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
auto pipe = Pipe::unitePipes(std::move(pipes));
ctx->rows_sources_read_buf->seek(0, 0);
auto transform = std::make_unique<ColumnGathererTransform>(pipe.getHeader(), pipe.numOutputPorts(), *ctx->rows_sources_read_buf);
const auto data_settings = global_ctx->data->getSettings();
auto transform = std::make_unique<ColumnGathererTransform>(
pipe.getHeader(),
pipe.numOutputPorts(),
*ctx->rows_sources_read_buf,
data_settings->merge_max_block_size,
data_settings->merge_max_block_size_bytes);
pipe.addTransform(std::move(transform));
ctx->column_parts_pipeline = QueryPipeline(std::move(pipe));

View File

@ -359,8 +359,7 @@ MergeTreeData::MergeTreeData(
, merging_params(merging_params_)
, require_part_metadata(require_part_metadata_)
, broken_part_callback(broken_part_callback_)
, log_name(std::make_shared<String>(table_id_.getNameForLogs()))
, log(getLogger(*log_name))
, log(table_id_.getNameForLogs())
, storage_settings(std::move(storage_settings_))
, pinned_part_uuids(std::make_shared<PinnedPartUUIDs>())
, data_parts_by_info(data_parts_indexes.get<TagByInfo>())
@ -1301,7 +1300,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
res.is_broken = true;
tryLogCurrentException(log, fmt::format("while loading part {} on path {}", part_name, part_path));
res.size_of_part = calculatePartSizeSafe(res.part, log);
res.size_of_part = calculatePartSizeSafe(res.part, log.load());
auto part_size_str = res.size_of_part ? formatReadableSizeWithBinarySuffix(*res.size_of_part) : "failed to calculate size";
LOG_ERROR(log,
@ -1332,7 +1331,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
if (part_disk_ptr->exists(marker_path))
{
/// NOTE: getBytesOnDisk() cannot be used here, since it may be zero if checksums.txt does not exist.
res.size_of_part = calculatePartSizeSafe(res.part, log);
res.size_of_part = calculatePartSizeSafe(res.part, log.load());
res.is_broken = true;
auto part_size_str = res.size_of_part ? formatReadableSizeWithBinarySuffix(*res.size_of_part) : "failed to calculate size";
@ -2119,7 +2118,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz
{
/// Actually we don't rely on temporary_directories_lifetime when removing old temporaries directories,
/// it's just an extra level of protection just in case we have a bug.
LOG_INFO(LogFrequencyLimiter(log, 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path);
LOG_INFO(LogFrequencyLimiter(log.load(), 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path);
continue;
}
else if (!disk->exists(it->path()))
@ -2739,8 +2738,7 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_
void MergeTreeData::renameInMemory(const StorageID & new_table_id)
{
IStorage::renameInMemory(new_table_id);
std::atomic_store(&log_name, std::make_shared<String>(new_table_id.getNameForLogs()));
log = getLogger(*log_name);
log.store(new_table_id.getNameForLogs());
}
void MergeTreeData::dropAllData()
@ -6263,13 +6261,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
log,
"Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found",
move_ttl_entry->destination_name,
*std::atomic_load(&log_name));
log.loadName());
else if (move_ttl_entry->destination_type == DataDestinationType::DISK && !move_ttl_entry->if_exists)
LOG_WARNING(
log,
"Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found",
move_ttl_entry->destination_name,
*std::atomic_load(&log_name));
log.loadName());
}
else if (is_insert && !perform_ttl_move_on_insert)
{
@ -6278,7 +6276,7 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
"TTL move on insert to {} {} for table {} is disabled",
(move_ttl_entry->destination_type == DataDestinationType::VOLUME ? "volume" : "disk"),
move_ttl_entry->destination_name,
*std::atomic_load(&log_name));
log.loadName());
}
else
{
@ -6294,13 +6292,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
log,
"Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space",
move_ttl_entry->destination_name,
*std::atomic_load(&log_name));
log.loadName());
else if (move_ttl_entry->destination_type == DataDestinationType::DISK)
LOG_WARNING(
log,
"Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space",
move_ttl_entry->destination_name,
*std::atomic_load(&log_name));
log.loadName());
}
}
}
@ -8005,7 +8003,7 @@ bool MergeTreeData::insertQueryIdOrThrowNoLock(const String & query_id, size_t m
throw Exception(
ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES,
"Too many simultaneous queries for table {}. Maximum is: {}",
*std::atomic_load(&log_name),
log.loadName(),
max_queries);
query_id_set.insert(query_id);
return true;
@ -8197,7 +8195,7 @@ ReservationPtr MergeTreeData::balancedReservation(
}
// Record submerging big parts in the tagger to clean them up.
tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log);
tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log.load());
}
}
}

View File

@ -462,7 +462,7 @@ public:
/// Load the set of data parts from disk. Call once - immediately after the object is created.
void loadDataParts(bool skip_sanity_checks, std::optional<std::unordered_set<std::string>> expected_parts);
String getLogName() const { return *std::atomic_load(&log_name); }
String getLogName() const { return log.loadName(); }
Int64 getMaxBlockNumber() const;
@ -1115,10 +1115,7 @@ protected:
/// Engine-specific methods
BrokenPartCallback broken_part_callback;
/// log_name will change during table RENAME. Use atomic_shared_ptr to allow concurrent RW.
/// NOTE clang-14 doesn't have atomic_shared_ptr yet. Use std::atomic* operations for now.
std::shared_ptr<String> log_name;
LoggerPtr log;
AtomicLogger log;
/// Storage settings.
/// Use get and set to receive readonly versions.

View File

@ -675,15 +675,25 @@ static NameToNameVector collectFilesForRenames(
{
if (command.type == MutationCommand::Type::DROP_INDEX)
{
if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx2"))
static const std::array<String, 2> suffixes = {".idx2", ".idx"};
static const std::array<String, 4> gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* is inverted index
for (const auto & suffix : suffixes)
{
add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx2", "");
add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, "");
const String filename = INDEX_FILE_PREFIX + command.column_name + suffix;
const String filename_mrk = INDEX_FILE_PREFIX + command.column_name + mrk_extension;
if (source_part->checksums.has(filename))
{
add_rename(filename, "");
add_rename(filename_mrk, "");
}
}
else if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx"))
for (const auto & gin_suffix : gin_suffixes)
{
add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx", "");
add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, "");
const String filename = INDEX_FILE_PREFIX + command.column_name + gin_suffix;
if (source_part->checksums.has(filename))
add_rename(filename, "");
}
}
else if (command.type == MutationCommand::Type::DROP_PROJECTION)

View File

@ -687,7 +687,7 @@ std::optional<MergeTreeMutationStatus> StorageMergeTree::getIncompleteMutationsS
const auto & mutation_entry = current_mutation_it->second;
auto txn = tryGetTransactionForMutation(mutation_entry, log);
auto txn = tryGetTransactionForMutation(mutation_entry, log.load());
/// There's no way a transaction may finish before a mutation that was started by the transaction.
/// But sometimes we need to check status of an unrelated mutation, in this case we don't care about transactions.
assert(txn || mutation_entry.tid.isPrehistoric() || from_another_mutation);
@ -835,7 +835,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
if (!to_kill)
return CancellationCode::NotFound;
if (auto txn = tryGetTransactionForMutation(*to_kill, log))
if (auto txn = tryGetTransactionForMutation(*to_kill, log.load()))
{
LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id);
TransactionLog::instance().rollbackTransaction(txn);
@ -1228,7 +1228,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid))
continue;
txn = tryGetTransactionForMutation(mutations_begin_it->second, log);
txn = tryGetTransactionForMutation(mutations_begin_it->second, log.load());
if (!txn)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} "
"that is going to be applied to part {}",

View File

@ -320,7 +320,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
attach,
[this] (const std::string & name) { enqueuePartForCheck(name); })
, zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_))
, zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log))
, zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log.load()))
, replica_name(replica_name_)
, replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_)
, reader(*this)
@ -812,7 +812,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
else
{
auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper);
if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log))
if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log.load()))
{
/// Someone is recursively removing table right now, we cannot create new table until old one is removed
continue;
@ -1130,7 +1130,7 @@ void StorageReplicatedMergeTree::drop()
if (lost_part_count > 0)
LOG_INFO(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count);
}
dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper);
dropReplica(zookeeper, zookeeper_path, replica_name, log.load(), getSettings(), &has_metadata_in_zookeeper);
}
}
@ -4183,7 +4183,7 @@ void StorageReplicatedMergeTree::startBeingLeader()
return;
}
zkutil::checkNoOldLeaders(log, *zookeeper, fs::path(zookeeper_path) / "leader_election");
zkutil::checkNoOldLeaders(log.load(), *zookeeper, fs::path(zookeeper_path) / "leader_election");
LOG_INFO(log, "Became leader");
is_leader = true;
@ -4277,7 +4277,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St
auto zookeeper = getZooKeeperIfTableShutDown();
auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log);
auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log.load());
if (unique_parts_set.empty())
{
LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts");
@ -9357,7 +9357,7 @@ StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, co
return unlockSharedDataByID(
part.getUniqueId(), shared_id, part.info, replica_name,
part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log, zookeeper_path, format_version);
part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log.load(), zookeeper_path, format_version);
}
namespace
@ -10310,7 +10310,7 @@ void StorageReplicatedMergeTree::backupData(
bool exists = false;
Strings mutation_ids;
{
ZooKeeperRetriesControl retries_ctl("getMutations", log, zookeeper_retries_info, nullptr);
ZooKeeperRetriesControl retries_ctl("getMutations", log.load(), zookeeper_retries_info, nullptr);
retries_ctl.retryLoop([&]()
{
if (!zookeeper || zookeeper->expired())
@ -10329,7 +10329,7 @@ void StorageReplicatedMergeTree::backupData(
bool mutation_id_exists = false;
String mutation;
ZooKeeperRetriesControl retries_ctl("getMutation", log, zookeeper_retries_info, nullptr);
ZooKeeperRetriesControl retries_ctl("getMutation", log.load(), zookeeper_retries_info, nullptr);
retries_ctl.retryLoop([&]()
{
if (!zookeeper || zookeeper->expired())

View File

@ -9,7 +9,8 @@
<users>
<readonly>
<password></password>
<networks incl="networks" replace="replace">
<networks>
<ip>::1</ip>
<ip>127.0.0.1</ip>
</networks>

View File

@ -18,7 +18,8 @@
<users>
<session_log_test_xml_user>
<no_password></no_password>
<networks incl="networks" replace="replace">
<networks>
<ip>::1</ip>
<ip>127.0.0.1</ip>
</networks>

View File

@ -0,0 +1,25 @@
SET allow_experimental_inverted_index = 1;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(
id UInt64,
str String,
INDEX idx str TYPE inverted(3) GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1;
INSERT INTO tab (str) VALUES ('I am inverted');
SELECT data_version FROM system.parts WHERE database = currentDatabase() AND table = 'tab' AND active = 1;
-- update column synchronously
ALTER TABLE tab UPDATE str = 'I am not inverted' WHERE 1 SETTINGS mutations_sync=1;
SELECT data_version FROM system.parts WHERE database = currentDatabase() AND table = 'tab' AND active = 1;
SELECT str FROM tab WHERE str LIKE '%inverted%' SETTINGS force_data_skipping_indices = 'idx';
DROP TABLE tab;

View File

@ -1,9 +1,20 @@
-- https://github.com/ClickHouse/ClickHouse/issues/52019
DROP TABLE IF EXISTS tab;
-- Test for Bug 52019: Undefined behavior
SET allow_experimental_inverted_index=1;
CREATE TABLE tab (`k` UInt64, `s` Map(String, String), INDEX af mapKeys(s) TYPE inverted(2) GRANULARITY 1) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab (
k UInt64,
s Map(String, String),
INDEX idx mapKeys(s) TYPE inverted(2) GRANULARITY 1)
ENGINE = MergeTree
ORDER BY k
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
INSERT INTO tab (k) VALUES (0);
SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1;
SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS allow_experimental_analyzer=1;
DROP TABLE tab;

View File

@ -0,0 +1,20 @@
-- This is supposed to test that DROP INDEX removes all index related files. Can't test this directly but at least run the statement and
-- check that no bad things happen.
SET allow_experimental_inverted_index = 1;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(
id UInt64,
doc String,
INDEX text_idx doc TYPE inverted
)
ENGINE = MergeTree
ORDER BY id
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;
ALTER TABLE tab DROP INDEX text_idx;
DROP TABLE tab;

View File

@ -2,8 +2,8 @@ SET allow_experimental_inverted_index = 1;
CREATE TABLE t
(
`key` UInt64,
`str` String,
key UInt64,
str String,
INDEX inv_idx str TYPE inverted(0) GRANULARITY 1
)
ENGINE = MergeTree
@ -13,4 +13,4 @@ INSERT INTO t VALUES (1, 'Hello World');
ALTER TABLE t DETACH PART 'all_1_1_0';
ALTER TABLE t ATTACH PART 'all_1_1_0';
ALTER TABLE t ATTACH PART 'all_1_1_0';

View File

@ -1,4 +1,7 @@
-- Tests that the inverted index can only be supported when allow_experimental_inverted_index = 1.
SET allow_experimental_inverted_index = 0;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(

View File

@ -1,3 +1,5 @@
-- Tests that match() utilizes the inverted index
SET allow_experimental_inverted_index = true;
DROP TABLE IF EXISTS tab;

View File

@ -1,25 +0,0 @@
SET allow_experimental_inverted_index=1;
DROP TABLE IF EXISTS t;
CREATE TABLE t
(
`timestamp` UInt64,
`s` String,
INDEX idx s TYPE inverted(3) GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1;
INSERT INTO t (s) VALUES ('I am inverted');
SELECT data_version FROM system.parts WHERE database=currentDatabase() AND table='t' AND active=1;
-- do update column synchronously
ALTER TABLE t UPDATE s='I am not inverted' WHERE 1 SETTINGS mutations_sync=1;
SELECT data_version FROM system.parts WHERE database=currentDatabase() AND table='t' AND active=1;
SELECT s FROM t WHERE s LIKE '%inverted%' SETTINGS force_data_skipping_indices='idx';
DROP TABLE t;

View File

@ -1,4 +1,4 @@
[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]]
[[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7]]
[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]]
[[53.946846,-4.8119445,43.525013,-23.71359,-42.472305,-51.636955,-50.458298,-51.982674,37.62072,-15.9006605,56.65076,-5.809669,57.143845,-2.0370207,54.050922,-4.897961,43.954018,-23.808758,-42.651337,-51.86827,-50.709732,-52.18156,37.734905,-15.853402,56.91643,-5.8815174,57.253094,-2.012879,54.157806,-4.9817176,44.384747,-23.902956,-42.830154,-52.10025,-50.96271,-52.3829,37.84573,-15.81032,57.177113,-5.958963,57.356136,-1.9952412,54.27533,-5.066312,44.878296,-23.956438,-42.993656,-52.337124,-51.208073,-52.615646,37.91102,-15.8062525,57.49891,-6.056076,57.45604,-1.9797823,54.39525,-5.1483474,45.374573],[88.028534,88.95315,89.87776,90.802376,91.64913,92.49588,93.342636,94.19737,95.0521,95.90684,96.712975,97.51912,98.32526,98.36342,98.40158,98.43974,98.36777,98.29579,98.223816,98.536446,98.849075,99.161705,99.7552,100.348694,100.94219,101.53184,102.12149,102.711136,103.79921,104.88729,105.975365,107.50462,109.033875,110.56313,111.79767,113.032196,114.26673,115.02128,115.775826,116.53037,117.15541,117.78044,118.40548,118.86489,119.3243,119.783714,120.04031,120.29691,120.55351,120.78621,121.01891,121.25161,121.533585,121.81555,122.09753,122.41821,122.7389,123.059586,123.39267],[-2.97538,2.8587952,-23.402771,0.91121674,4.8231735,9.141075,8.115662,10.785301,0.32717896,5.99382,-12.363731,5.29055,0.53089905,-2.3264008,-3.4524994,1.4582214,-2.321785,2.51297,5.4275208,3.3318253,5.8606567,0.019859314,-4.4901123,-12.495293,-5.8586197,-1.650322,-11.374588,4.3017426,4.042984,1.094429,9.639885,3.3983307,-3.20372,-5.462883,-5.834961,-6.649292,-1.1124649,3.7890396,16.047066,-2.5714111,8.488449,-2.785202,2.319191,-0.79857635,13.797401,-5.827278,-6.0466614,-5.9597855,-7.3454437,-3.1705627,6.0700684,3.5546417,1.9675064,-0.7594757,2.446434,0.5615692,0.86585236,-3.9112396,1.2327576]]
[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0],[10.1,20.449999,40.340004,10.100001,20.45,40.34,10.100001,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.100002,20.45,40.34]]
[[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7],[1.9999999,2,2,2,2,2,2,2,2,2,1.9999999,2,1.9999996,1.9999998]]
[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0],[10.1,20.449999,40.340004,10.100001,20.45,40.34,10.100001,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.100002,20.45,40.34]]
[[53.946846,-4.8119445,43.525013,-23.71359,-42.472305,-51.636955,-50.458298,-51.982674,37.62072,-15.9006605,56.65076,-5.809669,57.143845,-2.0370207,54.050922,-4.897961,43.954018,-23.808758,-42.651337,-51.86827,-50.709732,-52.18156,37.734905,-15.853402,56.91643,-5.8815174,57.253094,-2.012879,54.157806,-4.9817176,44.384747,-23.902956,-42.830154,-52.10025,-50.96271,-52.3829,37.84573,-15.81032,57.177113,-5.958963,57.356136,-1.9952412,54.27533,-5.066312,44.878296,-23.956438,-42.993656,-52.337124,-51.208073,-52.615646,37.91102,-15.8062525,57.49891,-6.056076,57.45604,-1.9797823,54.39525,-5.1483474,45.374573],[88.028534,88.95315,89.87776,90.802376,91.64913,92.49588,93.342636,94.19737,95.0521,95.90684,96.712975,97.51912,98.32526,98.36342,98.40158,98.43974,98.36777,98.29579,98.223816,98.536446,98.849075,99.161705,99.7552,100.348694,100.94219,101.53184,102.12149,102.711136,103.79921,104.88729,105.975365,107.50462,109.033875,110.56313,111.79767,113.032196,114.26673,115.02128,115.775826,116.53037,117.15541,117.78044,118.40548,118.86489,119.3243,119.783714,120.04031,120.29691,120.55351,120.78621,121.01891,121.25161,121.533585,121.81555,122.09753,122.41821,122.7389,123.059586,123.39267],[-2.97538,2.8587952,-23.402771,0.91121674,4.8231735,9.141075,8.115662,10.785301,0.32717896,5.99382,-12.363731,5.29055,0.53089905,-2.3264008,-3.4524994,1.4582214,-2.321785,2.51297,5.4275208,3.3318253,5.8606567,0.019859314,-4.4901123,-12.495293,-5.8586197,-1.650322,-11.374588,4.3017426,4.042984,1.094429,9.639885,3.3983307,-3.20372,-5.462883,-5.834961,-6.649292,-1.1124649,3.7890396,16.047066,-2.5714111,8.488449,-2.785202,2.319191,-0.79857635,13.797401,-5.827278,-6.0466614,-5.9597855,-7.3454437,-3.1705627,6.0700684,3.5546417,1.9675064,-0.7594757,2.446434,0.5615692,0.86585236,-3.9112396,1.2327576],[141.97537,84.141205,133.40277,67.08878,49.176826,40.858925,42.88434,42.2147,132.67282,80.00618,153.36374,91.70945,155.4691,96.3264,152.4525,93.54178,142.32178,74.48703,55.57248,46.668175,48.139343,46.980145,137.49011,84.49529,157.85863,95.65032,159.37459,100.69826,157.95702,99.90557,150.3601,83.60167,66.20372,58.462883,60.834957,60.649296,152.11246,99.21096,172.95294,110.57141,174.51155,115.7852,172.68082,113.79858,164.2026,95.82728,77.04666,67.95979,69.34544,68.17056,158.92993,105.44536,179.0325,115.759476,179.55356,120.43843,177.13416,117.91124,168.76724]]

View File

@ -0,0 +1,6 @@
CREATE TABLE test_table (i Int64) engine=MergeTree order by i;
CREATE DICTIONARY test_dict (y String, value UInt64 DEFAULT 0) PRIMARY KEY y SOURCE(CLICKHOUSE(TABLE 'test_table')) LAYOUT(DIRECT());
CREATE TABLE test_dict (y Int64) engine=MergeTree order by y; -- { serverError DICTIONARY_ALREADY_EXISTS }
CREATE DICTIONARY test_table (y String, value UInt64 DEFAULT 0) PRIMARY KEY y SOURCE(CLICKHOUSE(TABLE 'test_table')) LAYOUT(DIRECT()); -- { serverError TABLE_ALREADY_EXISTS }
CREATE DICTIONARY test_dict (y String, value UInt64 DEFAULT 0) PRIMARY KEY y SOURCE(CLICKHOUSE(TABLE 'test_table')) LAYOUT(DIRECT()); -- { serverError DICTIONARY_ALREADY_EXISTS }
CREATE TABLE test_table (y Int64) engine=MergeTree order by y; -- { serverError TABLE_ALREADY_EXISTS }

View File

@ -0,0 +1 @@
Vertical OK

View File

@ -0,0 +1,35 @@
-- Tags: long
DROP TABLE IF EXISTS t_vertical_merge_memory;
CREATE TABLE t_vertical_merge_memory (id UInt64, arr Array(String))
ENGINE = MergeTree ORDER BY id
SETTINGS
min_bytes_for_wide_part = 0,
vertical_merge_algorithm_min_rows_to_activate = 1,
vertical_merge_algorithm_min_columns_to_activate = 1,
index_granularity = 8192,
index_granularity_bytes = '10M',
merge_max_block_size = 8192,
merge_max_block_size_bytes = '10M';
INSERT INTO t_vertical_merge_memory SELECT number, arrayMap(x -> repeat('a', 50), range(1000)) FROM numbers(30000);
INSERT INTO t_vertical_merge_memory SELECT number, arrayMap(x -> repeat('a', 50), range(1000)) FROM numbers(30000);
OPTIMIZE TABLE t_vertical_merge_memory FINAL;
SYSTEM FLUSH LOGS;
SELECT
merge_algorithm,
peak_memory_usage < 500 * 1024 * 1024
? 'OK'
: format('FAIL: memory usage: {}', formatReadableSize(peak_memory_usage))
FROM system.part_log
WHERE
database = currentDatabase()
AND table = 't_vertical_merge_memory'
AND event_type = 'MergeParts'
AND length(merged_from) = 2;
DROP TABLE IF EXISTS t_vertical_merge_memory;

View File

@ -1,4 +1,4 @@
personal_ws-1.1 en 2657
personal_ws-1.1 en 2657
AArch
ACLs
ALTERs
@ -2060,7 +2060,6 @@ prebuild
prebuilt
preemptable
preferServerCiphers
prefertch
prefetch
prefetchsize
preloaded