mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 01:12:12 +00:00
Merge remote-tracking branch 'origin/master' into hive_text_parallel_parsing
This commit is contained in:
commit
0341880250
@ -328,23 +328,9 @@ if (ENABLE_BUILD_PROFILING)
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
if (${CMAKE_VERSION} VERSION_LESS "3.12.4")
|
||||
# CMake < 3.12 doesn't support setting 20 as a C++ standard version.
|
||||
# We will add C++ standard controlling flag in CMAKE_CXX_FLAGS manually for now.
|
||||
|
||||
if (COMPILER_GCC OR COMPILER_CLANG)
|
||||
# to make numeric_limits<__int128> works with GCC
|
||||
set (_CXX_STANDARD "gnu++2a")
|
||||
else ()
|
||||
set (_CXX_STANDARD "c++2a")
|
||||
endif ()
|
||||
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=${_CXX_STANDARD}")
|
||||
else ()
|
||||
set (CMAKE_CXX_STANDARD 20)
|
||||
set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html
|
||||
set (CMAKE_CXX_STANDARD_REQUIRED ON)
|
||||
endif ()
|
||||
set (CMAKE_CXX_STANDARD 20)
|
||||
set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html
|
||||
set (CMAKE_CXX_STANDARD_REQUIRED ON)
|
||||
|
||||
set (CMAKE_C_STANDARD 11)
|
||||
set (CMAKE_C_EXTENSIONS ON)
|
||||
|
@ -2,10 +2,6 @@
|
||||
option (ENABLE_CLANG_TIDY "Use clang-tidy static analyzer" OFF)
|
||||
|
||||
if (ENABLE_CLANG_TIDY)
|
||||
if (${CMAKE_VERSION} VERSION_LESS "3.6.0")
|
||||
message(FATAL_ERROR "clang-tidy requires CMake version at least 3.6.")
|
||||
endif()
|
||||
|
||||
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-13" "clang-tidy-12" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8")
|
||||
|
||||
if (CLANG_TIDY_PATH)
|
||||
|
@ -1,22 +1,15 @@
|
||||
if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64")
|
||||
if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386")
|
||||
message (FATAL_ERROR "32bit platforms are not supported")
|
||||
endif ()
|
||||
set (ARCH_AMD64 1)
|
||||
endif ()
|
||||
if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)")
|
||||
elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)")
|
||||
set (ARCH_AARCH64 1)
|
||||
endif ()
|
||||
if (ARCH_AARCH64 OR CMAKE_SYSTEM_PROCESSOR MATCHES "arm")
|
||||
set (ARCH_ARM 1)
|
||||
endif ()
|
||||
if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386")
|
||||
set (ARCH_I386 1)
|
||||
endif ()
|
||||
if ((ARCH_ARM AND NOT ARCH_AARCH64) OR ARCH_I386)
|
||||
message (FATAL_ERROR "32bit platforms are not supported")
|
||||
endif ()
|
||||
if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)")
|
||||
elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)")
|
||||
set (ARCH_PPC64LE 1)
|
||||
endif ()
|
||||
if (CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64")
|
||||
elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64")
|
||||
set (ARCH_RISCV64 1)
|
||||
else ()
|
||||
message (FATAL_ERROR "Platform ${CMAKE_SYSTEM_PROCESSOR} is not supported")
|
||||
endif ()
|
||||
|
||||
|
@ -13,7 +13,7 @@ execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version)
|
||||
if (COMPILER_GCC)
|
||||
# Require minimum version of gcc
|
||||
set (GCC_MINIMUM_VERSION 11)
|
||||
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION} AND NOT CMAKE_VERSION VERSION_LESS 2.8.9)
|
||||
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION})
|
||||
message (FATAL_ERROR "GCC version must be at least ${GCC_MINIMUM_VERSION}. For example, if GCC ${GCC_MINIMUM_VERSION} is available under gcc-${GCC_MINIMUM_VERSION}, g++-${GCC_MINIMUM_VERSION} names, do the following: export CC=gcc-${GCC_MINIMUM_VERSION} CXX=g++-${GCC_MINIMUM_VERSION}; rm -rf CMakeCache.txt CMakeFiles; and re run cmake or ./release.")
|
||||
endif ()
|
||||
|
||||
|
@ -17,7 +17,7 @@ services:
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir1:-}
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
- NET_ADMIN
|
||||
@ -47,7 +47,7 @@ services:
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir2:-}
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
- NET_ADMIN
|
||||
@ -77,7 +77,7 @@ services:
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir3:-}
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
- NET_ADMIN
|
||||
|
@ -75,7 +75,7 @@ This will create the `programs/clickhouse` executable, which can be used with `c
|
||||
The build requires the following components:
|
||||
|
||||
- Git (is used only to checkout the sources, it’s not needed for the build)
|
||||
- CMake 3.10 or newer
|
||||
- CMake 3.14 or newer
|
||||
- Ninja
|
||||
- C++ compiler: clang-13 or newer
|
||||
- Linker: lld
|
||||
|
@ -1,4 +1,4 @@
|
||||
# Integrating Rust libraries into ClickHouse.
|
||||
# Integrating Rust libraries
|
||||
|
||||
Rust library integration will be described based on BLAKE3 hash-function integration.
|
||||
|
||||
|
@ -13,6 +13,8 @@ Columns:
|
||||
- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
|
||||
- `query` (String) – The query text. For `INSERT`, it does not include the data to insert.
|
||||
- `query_id` (String) – Query ID, if defined.
|
||||
- `is_cancelled` (Int8) – Was query cancelled.
|
||||
- `is_all_data_sent` (Int8) – Was all data sent to the client (in other words query had been finished on the server).
|
||||
|
||||
```sql
|
||||
:) SELECT * FROM system.processes LIMIT 10 FORMAT Vertical;
|
||||
@ -43,6 +45,7 @@ http_user_agent:
|
||||
quota_key:
|
||||
elapsed: 0.000582537
|
||||
is_cancelled: 0
|
||||
is_all_data_sent: 0
|
||||
read_rows: 0
|
||||
read_bytes: 0
|
||||
total_rows_approx: 0
|
||||
|
@ -555,7 +555,7 @@ void LRUFileCache::remove(const Key & key)
|
||||
fs::remove(key_path);
|
||||
}
|
||||
|
||||
void LRUFileCache::tryRemoveAll()
|
||||
void LRUFileCache::remove(bool force_remove_unreleasable)
|
||||
{
|
||||
/// Try remove all cached files by cache_base_path.
|
||||
/// Only releasable file segments are evicted.
|
||||
@ -567,12 +567,13 @@ void LRUFileCache::tryRemoveAll()
|
||||
auto & [key, offset] = *it++;
|
||||
|
||||
auto * cell = getCell(key, offset, cache_lock);
|
||||
if (cell->releasable())
|
||||
if (cell->releasable() || force_remove_unreleasable)
|
||||
{
|
||||
auto file_segment = cell->file_segment;
|
||||
if (file_segment)
|
||||
{
|
||||
std::lock_guard<std::mutex> segment_lock(file_segment->mutex);
|
||||
file_segment->detached = true;
|
||||
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
|
||||
}
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
|
||||
virtual void remove(const Key & key) = 0;
|
||||
|
||||
virtual void tryRemoveAll() = 0;
|
||||
virtual void remove(bool force_remove_unreleasable) = 0;
|
||||
|
||||
static bool isReadOnly();
|
||||
|
||||
@ -145,7 +145,7 @@ public:
|
||||
|
||||
void remove(const Key & key) override;
|
||||
|
||||
void tryRemoveAll() override;
|
||||
void remove(bool force_remove_unreleasable) override;
|
||||
|
||||
std::vector<String> tryGetCachePaths(const Key & key) override;
|
||||
|
||||
|
@ -455,6 +455,8 @@ void FileSegment::complete(State state)
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
std::lock_guard segment_lock(mutex);
|
||||
|
||||
assertNotDetached();
|
||||
|
||||
bool is_downloader = isDownloaderImpl(segment_lock);
|
||||
if (!is_downloader)
|
||||
{
|
||||
@ -477,8 +479,6 @@ void FileSegment::complete(State state)
|
||||
|
||||
download_state = state;
|
||||
|
||||
assertNotDetached();
|
||||
|
||||
try
|
||||
{
|
||||
completeImpl(cache_lock, segment_lock);
|
||||
|
@ -52,7 +52,7 @@ private:
|
||||
{
|
||||
std::cerr << "Mutating\n";
|
||||
auto res = shallowMutate();
|
||||
res->wrapped = IColumn::mutate(wrapped);
|
||||
res->wrapped = IColumn::mutate(std::move(res->wrapped).detach());
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -54,9 +54,6 @@ static std::optional<Exception> checkTupleNames(const Strings & names)
|
||||
if (name.empty())
|
||||
return Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (isNumericASCII(name[0]))
|
||||
return Exception("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!names_set.insert(name).second)
|
||||
return Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <base/arithmeticOverflow.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
@ -20,6 +21,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
|
||||
@ -217,7 +219,9 @@ namespace
|
||||
{
|
||||
if (scale_multiplier < 1000)
|
||||
{
|
||||
Int64 t_milliseconds = t * (static_cast<Int64>(1000) / scale_multiplier);
|
||||
Int64 t_milliseconds = 0;
|
||||
if (common::mulOverflow(t, static_cast<Int64>(1000) / scale_multiplier, t_milliseconds))
|
||||
throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
if (likely(t >= 0))
|
||||
return t_milliseconds / milliseconds * milliseconds;
|
||||
else
|
||||
@ -252,7 +256,9 @@ namespace
|
||||
{
|
||||
if (scale_multiplier < 1000000)
|
||||
{
|
||||
Int64 t_microseconds = t * (static_cast<Int64>(1000000) / scale_multiplier);
|
||||
Int64 t_microseconds = 0;
|
||||
if (common::mulOverflow(t, static_cast<Int64>(1000000) / scale_multiplier, t_microseconds))
|
||||
throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
if (likely(t >= 0))
|
||||
return t_microseconds / microseconds * microseconds;
|
||||
else
|
||||
@ -287,7 +293,9 @@ namespace
|
||||
{
|
||||
if (scale_multiplier < 1000000000)
|
||||
{
|
||||
Int64 t_nanoseconds = t * (static_cast<Int64>(1000000000) / scale_multiplier);
|
||||
Int64 t_nanoseconds = 0;
|
||||
if (common::mulOverflow(t, (static_cast<Int64>(1000000000) / scale_multiplier), t_nanoseconds))
|
||||
throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
if (likely(t >= 0))
|
||||
return t_nanoseconds / nanoseconds * nanoseconds;
|
||||
else
|
||||
|
@ -306,12 +306,12 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
{
|
||||
auto caches = FileCacheFactory::instance().getAll();
|
||||
for (const auto & [_, cache_data] : caches)
|
||||
cache_data.cache->tryRemoveAll();
|
||||
cache_data.cache->remove(query.force_removal);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path);
|
||||
cache->tryRemoveAll();
|
||||
cache->remove(query.force_removal);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -455,6 +455,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
|
||||
res.client_info = client_info;
|
||||
res.elapsed_seconds = watch.elapsedSeconds();
|
||||
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
|
||||
res.is_all_data_sent = is_all_data_sent.load(std::memory_order_relaxed);
|
||||
res.read_rows = progress_in.read_rows;
|
||||
res.read_bytes = progress_in.read_bytes;
|
||||
res.total_rows = progress_in.total_rows_to_read;
|
||||
|
@ -61,6 +61,7 @@ struct QueryStatusInfo
|
||||
Int64 peak_memory_usage;
|
||||
ClientInfo client_info;
|
||||
bool is_cancelled;
|
||||
bool is_all_data_sent;
|
||||
|
||||
/// Optional fields, filled by query
|
||||
std::vector<UInt64> thread_ids;
|
||||
@ -101,6 +102,9 @@ protected:
|
||||
|
||||
std::atomic<bool> is_killed { false };
|
||||
|
||||
/// All data to the client already had been sent. Including EndOfStream.
|
||||
std::atomic<bool> is_all_data_sent { false };
|
||||
|
||||
void setUserProcessList(ProcessListForUser * user_process_list_);
|
||||
/// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently.
|
||||
const ProcessListForUser * getUserProcessList() const { return user_process_list; }
|
||||
@ -194,6 +198,9 @@ public:
|
||||
|
||||
bool isKilled() const { return is_killed; }
|
||||
|
||||
bool isAllDataSent() const { return is_all_data_sent; }
|
||||
void setAllDataSent() { is_all_data_sent = true; }
|
||||
|
||||
/// Adds a pipeline to the QueryStatus
|
||||
void addPipelineExecutor(PipelineExecutor * e);
|
||||
|
||||
|
@ -192,6 +192,13 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
<< (settings.hilite ? hilite_keyword : "") << " SECOND"
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else if (type == Type::DROP_FILESYSTEM_CACHE)
|
||||
{
|
||||
if (!filesystem_cache_path.empty())
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << filesystem_cache_path;
|
||||
if (force_removal)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORCE";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -89,7 +89,10 @@ public:
|
||||
String volume;
|
||||
String disk;
|
||||
UInt64 seconds{};
|
||||
|
||||
/// Values for `drop filesystem cache` system query.
|
||||
String filesystem_cache_path;
|
||||
bool force_removal = false;
|
||||
|
||||
String getID(char) const override { return "SYSTEM query"; }
|
||||
|
||||
|
@ -355,8 +355,9 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ASTPtr settings;
|
||||
|
||||
bool storage_like = false;
|
||||
bool parsed_engine_keyword = s_engine.ignore(pos, expected);
|
||||
|
||||
if (s_engine.ignore(pos, expected))
|
||||
if (parsed_engine_keyword)
|
||||
{
|
||||
s_eq.ignore(pos, expected);
|
||||
|
||||
@ -422,7 +423,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_settings.ignore(pos, expected))
|
||||
/// Do not allow SETTINGS clause without ENGINE,
|
||||
/// because we cannot distinguish engine settings from query settings in this case.
|
||||
/// And because settings for each engine are different.
|
||||
if (parsed_engine_keyword && s_settings.ignore(pos, expected))
|
||||
{
|
||||
if (!settings_p.parse(pos, settings, expected))
|
||||
return false;
|
||||
|
@ -346,6 +346,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
res->seconds = seconds->as<ASTLiteral>()->value.get<UInt64>();
|
||||
break;
|
||||
}
|
||||
case Type::DROP_FILESYSTEM_CACHE:
|
||||
{
|
||||
ParserLiteral path_parser;
|
||||
ASTPtr ast;
|
||||
if (path_parser.parse(pos, ast, expected))
|
||||
res->filesystem_cache_path = ast->as<ASTLiteral>()->value.safeGet<String>();
|
||||
if (ParserKeyword{"FORCE"}.ignore(pos, expected))
|
||||
res->force_removal = true;
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
{
|
||||
|
@ -98,6 +98,8 @@ void SortingStep::updateInputStream(DataStream input_stream)
|
||||
void SortingStep::updateOutputStream(Block result_header)
|
||||
{
|
||||
output_stream = createOutputStream(input_streams.at(0), std::move(result_header), getDataStreamTraits());
|
||||
output_stream->sort_description = result_description;
|
||||
output_stream->sort_mode = DataStream::SortMode::Stream;
|
||||
updateDistinctColumns(output_stream->header, output_stream->distinct_columns);
|
||||
}
|
||||
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include <Interpreters/InternalTextLogsQueue.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/Session.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
@ -1710,6 +1711,12 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace)
|
||||
void TCPHandler::sendEndOfStream()
|
||||
{
|
||||
state.sent_all_data = true;
|
||||
/// The following queries does not have process_list_entry:
|
||||
/// - internal
|
||||
/// - SHOW PROCESSLIST
|
||||
if (state.io.process_list_entry)
|
||||
(*state.io.process_list_entry)->setAllDataSent();
|
||||
|
||||
writeVarUInt(Protocol::Server::EndOfStream, *out);
|
||||
out->next();
|
||||
}
|
||||
|
@ -1414,6 +1414,9 @@ bool IMergeTreeDataPart::assertHasValidVersionMetadata() const
|
||||
if (part_is_probably_removed_from_disk)
|
||||
return true;
|
||||
|
||||
if (state == State::Temporary)
|
||||
return true;
|
||||
|
||||
DiskPtr disk = volume->getDisk();
|
||||
if (!disk->exists(getFullRelativePath()))
|
||||
return true;
|
||||
@ -1772,6 +1775,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix)
|
||||
void IMergeTreeDataPart::renameToDetached(const String & prefix) const
|
||||
{
|
||||
renameTo(getRelativePathForDetachedPart(prefix), true);
|
||||
part_is_probably_removed_from_disk = true;
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const
|
||||
|
@ -44,12 +44,11 @@ bool injectRequiredColumnsRecursively(
|
||||
if (alter_conversions.isColumnRenamed(column_name_in_part))
|
||||
column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part);
|
||||
|
||||
auto column_in_part = NameAndTypePair(
|
||||
column_name_in_part, column_in_storage->getSubcolumnName(),
|
||||
column_in_storage->getTypeInStorage(), column_in_storage->type);
|
||||
auto column_in_part = part->getColumns().tryGetByName(column_name_in_part);
|
||||
|
||||
/// column has files and hence does not require evaluation
|
||||
if (part->hasColumnFiles(column_in_part))
|
||||
if (column_in_part
|
||||
&& (!column_in_storage->isSubcolumn()
|
||||
|| column_in_part->type->tryGetSubcolumnType(column_in_storage->getSubcolumnName())))
|
||||
{
|
||||
/// ensure each column is added only once
|
||||
if (!required_columns.contains(column_name))
|
||||
|
@ -504,4 +504,15 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat
|
||||
return {new_part_name};
|
||||
}
|
||||
|
||||
String ReplicatedMergeTreeLogEntryData::getDescriptionForLogs(MergeTreeDataFormatVersion format_version) const
|
||||
{
|
||||
String description = fmt::format("{} with virtual parts [{}]", typeToString(), fmt::join(getVirtualPartNames(format_version), ", "));
|
||||
if (auto drop_range = getDropRange(format_version))
|
||||
{
|
||||
description += " and drop range ";
|
||||
description += *drop_range;
|
||||
}
|
||||
return description;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -144,6 +144,8 @@ struct ReplicatedMergeTreeLogEntryData
|
||||
/// Returns fake part for drop range (for DROP_RANGE and REPLACE_RANGE)
|
||||
std::optional<String> getDropRange(MergeTreeDataFormatVersion format_version) const;
|
||||
|
||||
String getDescriptionForLogs(MergeTreeDataFormatVersion format_version) const;
|
||||
|
||||
/// This entry is DROP PART, not DROP PARTITION. They both have same
|
||||
/// DROP_RANGE entry type, but differs in information about drop range.
|
||||
bool isDropPart(MergeTreeDataFormatVersion format_version) const;
|
||||
|
@ -213,7 +213,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
|
||||
{
|
||||
auto entry_virtual_parts = entry->getVirtualPartNames(format_version);
|
||||
|
||||
LOG_TEST(log, "Insert entry {} to queue with type {} with virtual parts [{}]", entry->znode_name, entry->typeToString(), fmt::join(entry_virtual_parts, ", "));
|
||||
LOG_TEST(log, "Insert entry {} to queue with type {}", entry->znode_name, entry->getDescriptionForLogs(format_version));
|
||||
|
||||
for (const String & virtual_part_name : entry_virtual_parts)
|
||||
{
|
||||
@ -293,9 +293,9 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
|
||||
{
|
||||
|
||||
auto entry_virtual_parts = entry->getVirtualPartNames(format_version);
|
||||
LOG_TEST(log, "Removing {} entry {} from queue with type {} with virtual parts [{}]",
|
||||
LOG_TEST(log, "Removing {} entry {} from queue with type {}",
|
||||
is_successful ? "successful" : "unsuccessful",
|
||||
entry->znode_name, entry->typeToString(), fmt::join(entry_virtual_parts, ", "));
|
||||
entry->znode_name, entry->getDescriptionForLogs(format_version));
|
||||
/// Update insert times.
|
||||
if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART)
|
||||
{
|
||||
|
@ -52,6 +52,7 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
|
||||
|
||||
{"elapsed", std::make_shared<DataTypeFloat64>()},
|
||||
{"is_cancelled", std::make_shared<DataTypeUInt8>()},
|
||||
{"is_all_data_sent", std::make_shared<DataTypeUInt8>()},
|
||||
{"read_rows", std::make_shared<DataTypeUInt64>()},
|
||||
{"read_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"total_rows_approx", std::make_shared<DataTypeUInt64>()},
|
||||
@ -120,6 +121,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c
|
||||
|
||||
res_columns[i++]->insert(process.elapsed_seconds);
|
||||
res_columns[i++]->insert(process.is_cancelled);
|
||||
res_columns[i++]->insert(process.is_all_data_sent);
|
||||
res_columns[i++]->insert(process.read_rows);
|
||||
res_columns[i++]->insert(process.read_bytes);
|
||||
res_columns[i++]->insert(process.total_rows);
|
||||
|
@ -225,6 +225,7 @@ def get_processlist_after_test(args):
|
||||
FROM clusterAllReplicas('test_cluster_database_replicated', system.processes)
|
||||
WHERE
|
||||
query NOT LIKE '%system.processes%' AND
|
||||
NOT is_all_data_sent AND
|
||||
Settings['log_comment'] = '{log_comment}' AND
|
||||
current_database = '{database}'
|
||||
""")
|
||||
@ -234,6 +235,7 @@ def get_processlist_after_test(args):
|
||||
FROM system.processes
|
||||
WHERE
|
||||
query NOT LIKE '%system.processes%' AND
|
||||
NOT is_all_data_sent AND
|
||||
Settings['log_comment'] = '{log_comment}' AND
|
||||
current_database = '{database}'
|
||||
""")
|
||||
|
@ -10,7 +10,18 @@
|
||||
<cache_enabled>0</cache_enabled>
|
||||
<data_cache_max_size>22548578304</data_cache_max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<data_cache_path>./s3_cache/</data_cache_path>
|
||||
</s3_cache>
|
||||
<s3_cache_2>
|
||||
<type>s3</type>
|
||||
<endpoint>http://localhost:11111/test/00170_test/</endpoint>
|
||||
<access_key_id>clickhouse</access_key_id>
|
||||
<secret_access_key>clickhouse</secret_access_key>
|
||||
<data_cache_enabled>1</data_cache_enabled>
|
||||
<cache_enabled>0</cache_enabled>
|
||||
<data_cache_max_size>22548578304</data_cache_max_size>
|
||||
<cache_on_write_operations>0</cache_on_write_operations>
|
||||
</s3_cache_2>
|
||||
</disks>
|
||||
<policies>
|
||||
<s3_cache>
|
||||
@ -20,6 +31,13 @@
|
||||
</main>
|
||||
</volumes>
|
||||
</s3_cache>
|
||||
<s3_cache_2>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>s3_cache_2</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</s3_cache_2>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
||||
|
@ -1,4 +1,5 @@
|
||||
<test>
|
||||
<query>SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number LIMIT 5</query>
|
||||
<query>SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number + 1 LIMIT 5</query>
|
||||
<query>SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number + 1 LIMIT 99999995, 5</query>
|
||||
</test>
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-backward-compatibility-check
|
||||
|
||||
DROP TABLE IF EXISTS alter_00061;
|
||||
CREATE TABLE alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192);
|
||||
|
||||
|
@ -17,9 +17,10 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE dst (n UInt64, type UInt8) ENGINE=Merge
|
||||
function thread_insert()
|
||||
{
|
||||
set -e
|
||||
trap "exit 0" INT
|
||||
val=1
|
||||
while true; do
|
||||
trap "STOP_THE_LOOP=1" INT
|
||||
STOP_THE_LOOP=0
|
||||
while [[ $STOP_THE_LOOP != 1 ]]; do
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
BEGIN TRANSACTION;
|
||||
INSERT INTO src VALUES /* ($val, 1) */ ($val, 1);
|
||||
@ -91,8 +92,9 @@ function thread_partition_dst_to_src()
|
||||
function thread_select()
|
||||
{
|
||||
set -e
|
||||
trap "exit 0" INT
|
||||
while true; do
|
||||
trap "STOP_THE_LOOP=1" INT
|
||||
STOP_THE_LOOP=0
|
||||
while [[ $STOP_THE_LOOP != 1 ]]; do
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
BEGIN TRANSACTION;
|
||||
-- no duplicates
|
||||
|
@ -50,8 +50,9 @@ function thread_insert_rollback()
|
||||
function thread_optimize()
|
||||
{
|
||||
set -e
|
||||
trap "exit 0" INT
|
||||
while true; do
|
||||
trap "STOP_THE_LOOP=1" INT
|
||||
STOP_THE_LOOP=0
|
||||
while [[ $STOP_THE_LOOP != 1 ]]; do
|
||||
optimize_query="OPTIMIZE TABLE src"
|
||||
partition_id=$(( RANDOM % 2 ))
|
||||
if (( RANDOM % 2 )); then
|
||||
@ -102,8 +103,9 @@ function thread_select()
|
||||
function thread_select_insert()
|
||||
{
|
||||
set -e
|
||||
trap "exit 0" INT
|
||||
while true; do
|
||||
trap "STOP_THE_LOOP=1" INT
|
||||
STOP_THE_LOOP=0
|
||||
while [[ $STOP_THE_LOOP != 1 ]]; do
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
BEGIN TRANSACTION;
|
||||
SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null;
|
||||
|
@ -35,8 +35,9 @@ function thread_insert_rollback()
|
||||
|
||||
function thread_select()
|
||||
{
|
||||
trap "exit 0" INT
|
||||
while true; do
|
||||
trap "STOP_THE_LOOP=1" INT
|
||||
STOP_THE_LOOP=0
|
||||
while [[ $STOP_THE_LOOP != 1 ]]; do
|
||||
# Result of `uniq | wc -l` must be 1 if the first and the last queries got the same result
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
BEGIN TRANSACTION;
|
||||
|
@ -7,6 +7,6 @@ INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (2, 11, 4395), (3, 22, 3545), (4,
|
||||
SET max_block_size = 1;
|
||||
SET max_rows_to_read = 5;
|
||||
|
||||
SELECT toUInt32(x), y, z FROM pk WHERE (x >= toDateTime(100000)) AND (x <= toDateTime(3));
|
||||
SELECT toUInt32(x), y, z FROM pk WHERE (x >= toDateTime(100000)) AND (x <= toDateTime(90000));
|
||||
|
||||
DROP TABLE IF EXISTS pk;
|
||||
|
@ -33,13 +33,12 @@ Expression (Projection)
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Expression (Projection)
|
||||
Expression ((Projection + Before ORDER BY [lifted up part]))
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Expression (Before ORDER BY [lifted up part])
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree (default.test_table)
|
||||
optimize_aggregation_in_order
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Aggregating
|
||||
|
@ -144,7 +144,7 @@ Filter
|
||||
2 3
|
||||
> function calculation should be done after sorting and limit (if possible)
|
||||
> Expression should be divided into two subexpressions and only one of them should be moved after Sorting
|
||||
Expression (Before ORDER BY [lifted up part])
|
||||
Expression ((Projection + Before ORDER BY [lifted up part]))
|
||||
FUNCTION sipHash64
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
|
@ -201,7 +201,7 @@ echo "> function calculation should be done after sorting and limit (if possible
|
||||
echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" |
|
||||
sed 's/^ *//g' | grep -o "^ *\(Expression (Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)"
|
||||
sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)"
|
||||
echo "> this query should be executed without throwing an exception"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select throwIf(number = 5) from (select * from numbers(10)) order by number limit 1"
|
||||
|
@ -0,0 +1,3 @@
|
||||
{"data":{"k1":0,"k2":2}}
|
||||
{"data":{"k1":1,"k2":0}}
|
||||
Tuple(k1 Int8, k2 Int8)
|
@ -0,0 +1,25 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
SET output_format_json_named_tuples_as_objects = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_json_wide_parts;
|
||||
|
||||
CREATE TABLE t_json_wide_parts (data JSON)
|
||||
ENGINE MergeTree ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
SYSTEM STOP MERGES t_json_wide_parts;
|
||||
|
||||
INSERT INTO t_json_wide_parts VALUES ('{"k1": 1}');
|
||||
INSERT INTO t_json_wide_parts VALUES ('{"k2": 2}');
|
||||
|
||||
SYSTEM START MERGES t_json_wide_parts;
|
||||
OPTIMIZE TABLE t_json_wide_parts FINAL;
|
||||
|
||||
SELECT data FROM t_json_wide_parts ORDER BY data.k1 FORMAT JSONEachRow;
|
||||
|
||||
SELECT type FROM system.parts_columns
|
||||
WHERE table = 't_json_wide_parts' AND database = currentDatabase() AND active;
|
||||
|
||||
DROP TABLE t_json_wide_parts;
|
@ -1 +1,2 @@
|
||||
Ok
|
||||
SleepFunctionCalls: 4 (increment)
|
||||
SleepFunctionMicroseconds: 400000 (increment)
|
||||
|
@ -7,22 +7,16 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
${CLICKHOUSE_CLIENT} --multiquery --query "
|
||||
drop table if exists aliases_lazyness;
|
||||
create table aliases_lazyness (x UInt32, y ALIAS sleepEachRow(0.1)) Engine=MergeTree ORDER BY x;
|
||||
insert into aliases_lazyness(x) select * from numbers(40);
|
||||
insert into aliases_lazyness(x) select * from numbers(100);
|
||||
"
|
||||
|
||||
# In very old ClickHouse versions alias column was calculated for every row.
|
||||
# If it works this way, the query will take at least 0.1 * 40 = 4 seconds.
|
||||
# If the issue does not exist, the query should take slightly more than 0.1 seconds.
|
||||
# The exact time is not guaranteed, so we check in a loop that at least once
|
||||
# the query will process in less than one second, that proves that the behaviour is not like it was long time ago.
|
||||
# If it works this way, the query will take at least 0.1 * 100 = 10 seconds.
|
||||
# If the issue does not exist, the query should call sleepEachRow() "only" 4 times:
|
||||
# - from MergeTreeData::getQueryProcessingStageWithAggregateProjection() -> MergeTreeWhereOptimizer -> getBlockWithConstants()
|
||||
# - from MergeTreeWhereOptimizer -> getBlockWithConstants()
|
||||
# - ReadFromMergeTree::selectRangesToRead() -> getBlockWithConstants()
|
||||
# - Pipeline
|
||||
${CLICKHOUSE_CLIENT} --profile-events-delay-ms=-1 --print-profile-events --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" |& grep -o -e "SleepFunctionMicroseconds.*" -e "SleepFunctionCalls.*"
|
||||
|
||||
i=0 retries=300
|
||||
while [[ $i -lt $retries ]]; do
|
||||
timeout 1 ${CLICKHOUSE_CLIENT} --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" && break
|
||||
((++i))
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --multiquery --query "
|
||||
drop table aliases_lazyness;
|
||||
SELECT 'Ok';
|
||||
"
|
||||
${CLICKHOUSE_CLIENT} --query "drop table aliases_lazyness"
|
||||
|
File diff suppressed because one or more lines are too long
@ -16,69 +16,69 @@ tables_with_database_column=(
|
||||
|
||||
*/
|
||||
use system;
|
||||
show create table aggregate_function_combinators;
|
||||
show create table asynchronous_inserts;
|
||||
show create table asynchronous_metrics;
|
||||
show create table build_options;
|
||||
show create table clusters;
|
||||
show create table collations;
|
||||
show create table columns;
|
||||
show create table contributors;
|
||||
show create table current_roles;
|
||||
show create table data_skipping_indices;
|
||||
show create table data_type_families;
|
||||
show create table databases;
|
||||
show create table detached_parts;
|
||||
show create table dictionaries;
|
||||
show create table disks;
|
||||
show create table distributed_ddl_queue;
|
||||
show create table distribution_queue;
|
||||
show create table enabled_roles;
|
||||
show create table errors;
|
||||
show create table events;
|
||||
show create table formats;
|
||||
show create table functions;
|
||||
show create table grants;
|
||||
show create table graphite_retentions;
|
||||
show create table licenses;
|
||||
show create table macros;
|
||||
show create table merge_tree_settings;
|
||||
show create table merges;
|
||||
show create table metrics;
|
||||
show create table models;
|
||||
show create table mutations;
|
||||
show create table numbers;
|
||||
show create table numbers_mt;
|
||||
show create table one;
|
||||
show create table part_moves_between_shards;
|
||||
show create table parts;
|
||||
show create table parts_columns;
|
||||
show create table privileges;
|
||||
show create table processes;
|
||||
show create table projection_parts;
|
||||
show create table projection_parts_columns;
|
||||
show create table quota_limits;
|
||||
show create table quota_usage;
|
||||
show create table quotas;
|
||||
show create table quotas_usage;
|
||||
show create table replicas;
|
||||
show create table replicated_fetches;
|
||||
show create table replicated_merge_tree_settings;
|
||||
show create table replication_queue;
|
||||
show create table role_grants;
|
||||
show create table roles;
|
||||
show create table row_policies;
|
||||
show create table settings;
|
||||
show create table settings_profile_elements;
|
||||
show create table settings_profiles;
|
||||
show create table stack_trace;
|
||||
show create table storage_policies;
|
||||
show create table table_engines;
|
||||
show create table table_functions;
|
||||
show create table tables;
|
||||
show create table time_zones;
|
||||
show create table user_directories;
|
||||
show create table users;
|
||||
show create table warnings;
|
||||
show create table zeros;
|
||||
show create table zeros_mt;
|
||||
show create table aggregate_function_combinators format TSVRaw;
|
||||
show create table asynchronous_inserts format TSVRaw;
|
||||
show create table asynchronous_metrics format TSVRaw;
|
||||
show create table build_options format TSVRaw;
|
||||
show create table clusters format TSVRaw;
|
||||
show create table collations format TSVRaw;
|
||||
show create table columns format TSVRaw;
|
||||
show create table contributors format TSVRaw;
|
||||
show create table current_roles format TSVRaw;
|
||||
show create table data_skipping_indices format TSVRaw;
|
||||
show create table data_type_families format TSVRaw;
|
||||
show create table databases format TSVRaw;
|
||||
show create table detached_parts format TSVRaw;
|
||||
show create table dictionaries format TSVRaw;
|
||||
show create table disks format TSVRaw;
|
||||
show create table distributed_ddl_queue format TSVRaw;
|
||||
show create table distribution_queue format TSVRaw;
|
||||
show create table enabled_roles format TSVRaw;
|
||||
show create table errors format TSVRaw;
|
||||
show create table events format TSVRaw;
|
||||
show create table formats format TSVRaw;
|
||||
show create table functions format TSVRaw;
|
||||
show create table grants format TSVRaw;
|
||||
show create table graphite_retentions format TSVRaw;
|
||||
show create table licenses format TSVRaw;
|
||||
show create table macros format TSVRaw;
|
||||
show create table merge_tree_settings format TSVRaw;
|
||||
show create table merges format TSVRaw;
|
||||
show create table metrics format TSVRaw;
|
||||
show create table models format TSVRaw;
|
||||
show create table mutations format TSVRaw;
|
||||
show create table numbers format TSVRaw;
|
||||
show create table numbers_mt format TSVRaw;
|
||||
show create table one format TSVRaw;
|
||||
show create table part_moves_between_shards format TSVRaw;
|
||||
show create table parts format TSVRaw;
|
||||
show create table parts_columns format TSVRaw;
|
||||
show create table privileges format TSVRaw;
|
||||
show create table processes format TSVRaw;
|
||||
show create table projection_parts format TSVRaw;
|
||||
show create table projection_parts_columns format TSVRaw;
|
||||
show create table quota_limits format TSVRaw;
|
||||
show create table quota_usage format TSVRaw;
|
||||
show create table quotas format TSVRaw;
|
||||
show create table quotas_usage format TSVRaw;
|
||||
show create table replicas format TSVRaw;
|
||||
show create table replicated_fetches format TSVRaw;
|
||||
show create table replicated_merge_tree_settings format TSVRaw;
|
||||
show create table replication_queue format TSVRaw;
|
||||
show create table role_grants format TSVRaw;
|
||||
show create table roles format TSVRaw;
|
||||
show create table row_policies format TSVRaw;
|
||||
show create table settings format TSVRaw;
|
||||
show create table settings_profile_elements format TSVRaw;
|
||||
show create table settings_profiles format TSVRaw;
|
||||
show create table stack_trace format TSVRaw;
|
||||
show create table storage_policies format TSVRaw;
|
||||
show create table table_engines format TSVRaw;
|
||||
show create table table_functions format TSVRaw;
|
||||
show create table tables format TSVRaw;
|
||||
show create table time_zones format TSVRaw;
|
||||
show create table user_directories format TSVRaw;
|
||||
show create table users format TSVRaw;
|
||||
show create table warnings format TSVRaw;
|
||||
show create table zeros format TSVRaw;
|
||||
show create table zeros_mt format TSVRaw;
|
||||
|
@ -7,15 +7,13 @@
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
Limit
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Sorting)
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
(Sorting)
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
@ -25,18 +23,16 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
Limit
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Sorting)
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
(Sorting)
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
@ -46,17 +42,15 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
Limit
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Sorting)
|
||||
FinishSortingTransform
|
||||
PartialSortingTransform
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
(Sorting)
|
||||
FinishSortingTransform
|
||||
PartialSortingTransform
|
||||
MergingSortedTransform 2 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(SettingQuotaAndLimits)
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
2020-10-11 0
|
||||
2020-10-11 0
|
||||
2020-10-11 0
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-random-settings
|
||||
|
||||
create table if not exists t_group_by_lowcardinality(p_date Date, val LowCardinality(Nullable(String)))
|
||||
engine=MergeTree() partition by p_date order by tuple();
|
||||
|
||||
|
@ -25,3 +25,5 @@ CREATE TABLE default.mem\n(\n `n` UInt8\n)\nENGINE = Memory
|
||||
CREATE TABLE default.val\n(\n `n` Int32\n) AS values(\'n int\', 1, 2)
|
||||
CREATE TABLE default.val2\n(\n `n` Int32\n) AS values(\'n int\', 1, 2)
|
||||
CREATE TABLE default.log\n(\n `n` Int32\n)\nENGINE = Log
|
||||
CREATE TABLE default.kek\n(\n `n` Int32\n)\nENGINE = Memory
|
||||
CREATE TABLE default.lol\n(\n `n` Int32\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS min_bytes_for_wide_part = 123, index_granularity = 8192
|
||||
|
@ -66,7 +66,7 @@ DROP TABLE t2;
|
||||
|
||||
|
||||
CREATE DATABASE test_02184 ORDER BY kek; -- {serverError 80}
|
||||
CREATE DATABASE test_02184 SETTINGS x=1; -- {serverError 80}
|
||||
CREATE DATABASE test_02184 SETTINGS x=1; -- {serverError 115}
|
||||
CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) ENGINE=MergeTree PRIMARY KEY y; -- {clientError 36}
|
||||
SET default_table_engine = 'MergeTree';
|
||||
CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) PRIMARY KEY y; -- {clientError 36}
|
||||
@ -118,3 +118,13 @@ SHOW CREATE TABLE log;
|
||||
DROP TABLE val;
|
||||
DROP TABLE val2;
|
||||
DROP TABLE log;
|
||||
|
||||
DROP TABLE IF EXISTS kek;
|
||||
DROP TABLE IF EXISTS lol;
|
||||
SET default_table_engine = 'Memory';
|
||||
CREATE TABLE kek (n int) SETTINGS log_queries=1;
|
||||
CREATE TABLE lol (n int) ENGINE=MergeTree ORDER BY n SETTINGS min_bytes_for_wide_part=123 SETTINGS log_queries=1;
|
||||
SHOW CREATE TABLE kek;
|
||||
SHOW CREATE TABLE lol;
|
||||
DROP TABLE kek;
|
||||
DROP TABLE lol;
|
||||
|
@ -0,0 +1,6 @@
|
||||
select toStartOfInterval(toDateTime64('\0930-12-12 12:12:12.1234567', 3), toIntervalNanosecond(1024)); -- {serverError 407}
|
||||
|
||||
SELECT
|
||||
toDateTime64(-9223372036854775808, 1048575, toIntervalNanosecond(9223372036854775806), NULL),
|
||||
toStartOfInterval(toDateTime64(toIntervalNanosecond(toIntervalNanosecond(257), toDateTime64(toStartOfInterval(toDateTime64(NULL)))), '', 100), toIntervalNanosecond(toStartOfInterval(toDateTime64(toIntervalNanosecond(NULL), NULL)), -1)),
|
||||
toStartOfInterval(toDateTime64('\0930-12-12 12:12:12.1234567', 3), toIntervalNanosecond(1024)); -- {serverError 407}
|
@ -0,0 +1,30 @@
|
||||
174876E800
|
||||
174876E800
|
||||
64
|
||||
9C
|
||||
64
|
||||
FF9C
|
||||
64
|
||||
FFFFFF9C
|
||||
64
|
||||
FFFFFFFFFFFFFF9C
|
||||
174876E800
|
||||
174876E800
|
||||
174876E800
|
||||
FFFFFFFFFFFFFFFFFFFFFFE8B7891800
|
||||
FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE8B7891800
|
||||
0001011101001000011101101110100000000000
|
||||
0001011101001000011101101110100000000000
|
||||
01100100
|
||||
10011100
|
||||
01100100
|
||||
1111111110011100
|
||||
01100100
|
||||
11111111111111111111111110011100
|
||||
01100100
|
||||
1111111111111111111111111111111111111111111111111111111110011100
|
||||
0001011101001000011101101110100000000000
|
||||
0001011101001000011101101110100000000000
|
||||
0001011101001000011101101110100000000000
|
||||
11111111111111111111111111111111111111111111111111111111111111111111111111111111111111111110100010110111100010010001100000000000
|
||||
1111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111110100010110111100010010001100000000000
|
@ -28,3 +28,33 @@ SELECT hex(toInt128(-100000000000)) AS res;
|
||||
|
||||
SELECT hex(toInt256(-100000000000)) AS res;
|
||||
|
||||
SELECT bin(toUInt128(100000000000)) AS res;
|
||||
|
||||
SELECT bin(toUInt256(100000000000)) AS res;
|
||||
|
||||
SELECT bin(toInt8(100)) AS res;
|
||||
|
||||
SELECT bin(toInt8(-100)) AS res;
|
||||
|
||||
SELECT bin(toInt16(100)) AS res;
|
||||
|
||||
SELECT bin(toInt16(-100)) AS res;
|
||||
|
||||
SELECT bin(toInt32(100)) AS res;
|
||||
|
||||
SELECT bin(toInt32(-100)) AS res;
|
||||
|
||||
SELECT bin(toInt64(100)) AS res;
|
||||
|
||||
SELECT bin(toInt64(-100)) AS res;
|
||||
|
||||
SELECT bin(toInt128(100000000000)) AS res;
|
||||
|
||||
SELECT bin(toInt128(100000000000)) AS res;
|
||||
|
||||
SELECT bin(toInt256(100000000000)) AS res;
|
||||
|
||||
SELECT bin(toInt128(-100000000000)) AS res;
|
||||
|
||||
SELECT bin(toInt256(-100000000000)) AS res;
|
||||
|
@ -1,15 +0,0 @@
|
||||
174876E800
|
||||
174876E800
|
||||
64
|
||||
9C
|
||||
64
|
||||
FF9C
|
||||
64
|
||||
FFFFFF9C
|
||||
64
|
||||
FFFFFFFFFFFFFF9C
|
||||
174876E800
|
||||
174876E800
|
||||
174876E800
|
||||
FFFFFFFFFFFFFFFFFFFFFFE8B7891800
|
||||
FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE8B7891800
|
@ -0,0 +1,30 @@
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
0
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
2
|
||||
SYSTEM DROP FILESYSTEM CACHE FORCE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
0
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
1
|
||||
SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 }
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
1
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
3
|
||||
SYSTEM DROP FILESYSTEM CACHE './s3_cache/';
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
2
|
34
tests/queries/0_stateless/02286_drop_filesystem_cache.sql
Normal file
34
tests/queries/0_stateless/02286_drop_filesystem_cache.sql
Normal file
@ -0,0 +1,34 @@
|
||||
-- Tags: no-parallel, no-fasttest, no-s3-storage
|
||||
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE FORCE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 }
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE './s3_cache/';
|
||||
SELECT count() FROM system.filesystem_cache;
|
@ -0,0 +1,10 @@
|
||||
CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` Int32), `4` Int32)\n)\nENGINE = Memory
|
||||
{"t":{"1":{"2":2,"3":3},"4":4}}
|
||||
2 3 4
|
||||
2 3 4
|
||||
2 3 4
|
||||
2 3 4
|
||||
Tuple(`1` Tuple(`2` Int8, `3` Int8), `4` Int8)
|
||||
{"t":{"1":{"2":2,"3":3},"4":4}}
|
||||
2 3 4
|
||||
(('value'))
|
45
tests/queries/0_stateless/02286_tuple_numeric_identifier.sql
Normal file
45
tests/queries/0_stateless/02286_tuple_numeric_identifier.sql
Normal file
@ -0,0 +1,45 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS t_tuple_numeric;
|
||||
|
||||
CREATE TABLE t_tuple_numeric (t Tuple(`1` Tuple(`2` Int, `3` Int), `4` Int)) ENGINE = Memory;
|
||||
SHOW CREATE TABLE t_tuple_numeric;
|
||||
|
||||
INSERT INTO t_tuple_numeric VALUES (((2, 3), 4));
|
||||
|
||||
SET output_format_json_named_tuples_as_objects = 1;
|
||||
|
||||
SELECT * FROM t_tuple_numeric FORMAT JSONEachRow;
|
||||
SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric;
|
||||
SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric;
|
||||
|
||||
SELECT t.1.3 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX}
|
||||
SELECT t.4 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX}
|
||||
SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; -- {serverError UNKNOWN_IDENTIFIER}
|
||||
|
||||
DROP TABLE t_tuple_numeric;
|
||||
|
||||
CREATE TABLE t_tuple_numeric (t Tuple(Tuple(Int, Int), Int)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO t_tuple_numeric VALUES (((2, 3), 4));
|
||||
|
||||
SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric;
|
||||
SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric;
|
||||
|
||||
DROP TABLE t_tuple_numeric;
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
CREATE TABLE t_tuple_numeric (t JSON) ENGINE = Memory;
|
||||
INSERT INTO t_tuple_numeric FORMAT JSONEachRow {"t":{"1":{"2":2,"3":3},"4":4}}
|
||||
|
||||
SELECT toTypeName(t) FROM t_tuple_numeric LIMIT 1;
|
||||
|
||||
SELECT * FROM t_tuple_numeric FORMAT JSONEachRow;
|
||||
SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric;
|
||||
|
||||
DROP TABLE t_tuple_numeric;
|
||||
|
||||
WITH
|
||||
'{"1":{"key":"value"}}' AS data,
|
||||
JSONExtract(data, 'Tuple("1" Tuple(key String))') AS parsed_json
|
||||
SELECT parsed_json AS ssid
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel
|
||||
-- Tags: no-parallel, no-random-settings
|
||||
|
||||
-- { echo }
|
||||
|
||||
|
@ -4,3 +4,4 @@ requests
|
||||
sqlparse
|
||||
tenacity
|
||||
xmltodict
|
||||
pandas
|
||||
|
@ -1,4 +1,6 @@
|
||||
v22.4.3.3-stable 2022-04-26
|
||||
v22.4.2.1-stable 2022-04-22
|
||||
v22.3.4.20-lts 2022-04-26
|
||||
v22.3.3.44-lts 2022-04-06
|
||||
v22.3.2.2-lts 2022-03-17
|
||||
v22.2.3.5-stable 2022-02-25
|
||||
|
|
Loading…
Reference in New Issue
Block a user