Merge branch 'master' into allow-parallel-replicas-for-join-with-analyzer-2

This commit is contained in:
Nikolai Kochetov 2024-02-07 12:25:11 +01:00 committed by GitHub
commit c434748fc1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
85 changed files with 955 additions and 505 deletions

View File

@ -17,7 +17,7 @@ assignees: ''
> A link to reproducer in [https://fiddle.clickhouse.com/](https://fiddle.clickhouse.com/).
**Does it reproduce on recent release?**
**Does it reproduce on the most recent release?**
[The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv)
@ -34,11 +34,11 @@ assignees: ''
**How to reproduce**
* Which ClickHouse server version to use
* Which interface to use, if matters
* Which interface to use, if it matters
* Non-default settings, if any
* `CREATE TABLE` statements for all tables involved
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/ClickHouse/ClickHouse/blob/master/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
* Queries to run that lead to unexpected result
* Queries to run that lead to an unexpected result
**Expected behavior**

View File

@ -55,7 +55,6 @@ jobs:
uses: ./.github/workflows/reusable_docker.yml
with:
data: ${{ needs.RunConfig.outputs.data }}
set_latest: true
StyleCheck:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
@ -362,14 +361,6 @@ jobs:
test_name: Stateless tests (release)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestReleaseDatabaseOrdinary:
needs: [RunConfig, BuilderDebRelease]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Stateless tests (release, DatabaseOrdinary)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestReleaseDatabaseReplicated:
needs: [RunConfig, BuilderDebRelease]
if: ${{ !failure() && !cancelled() }}
@ -733,7 +724,6 @@ jobs:
- MarkReleaseReady
- FunctionalStatelessTestDebug
- FunctionalStatelessTestRelease
- FunctionalStatelessTestReleaseDatabaseOrdinary
- FunctionalStatelessTestReleaseDatabaseReplicated
- FunctionalStatelessTestReleaseAnalyzer
- FunctionalStatelessTestReleaseS3

View File

@ -28,7 +28,7 @@ jobs:
id: runconfig
run: |
echo "::group::configure CI run"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --skip-jobs --rebuild-all-docker --outfile ${{ runner.temp }}/ci_run_data.json
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --skip-jobs --outfile ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
echo "::group::CI run configure results"

View File

@ -46,7 +46,7 @@ jobs:
needs: [DockerBuildAmd64, DockerBuildAarch64]
runs-on: [self-hosted, style-checker]
if: |
!failure() && !cancelled() && toJson(fromJson(inputs.data).docker_data.missing_multi) != '[]'
!failure() && !cancelled() && (toJson(fromJson(inputs.data).docker_data.missing_multi) != '[]' || inputs.set_latest)
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
@ -55,14 +55,12 @@ jobs:
- name: Build images
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
FLAG_LATEST=''
if [ "${{ inputs.set_latest }}" == "true" ]; then
FLAG_LATEST='--set-latest'
echo "latest tag will be set for resulting manifests"
python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 \
--image-tags '${{ toJson(fromJson(inputs.data).docker_data.images) }}' \
--missing-images '${{ toJson(fromJson(inputs.data).docker_data.missing_multi) }}' \
--set-latest
else
python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 \
--image-tags '${{ toJson(fromJson(inputs.data).docker_data.images) }}' \
--missing-images '${{ toJson(fromJson(inputs.data).docker_data.missing_multi) }}'
fi
python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 \
--image-tags '${{ toJson(fromJson(inputs.data).docker_data.images) }}' \
--missing-images '${{ toJson(fromJson(inputs.data).docker_data.missing_multi) }}' \
$FLAG_LATEST

View File

@ -22,7 +22,6 @@
* Add `quantileDD` aggregate function as well as the corresponding `quantilesDD` and `medianDD`. It is based on the DDSketch https://www.vldb.org/pvldb/vol12/p2195-masson.pdf. ### Documentation entry for user-facing changes. [#56342](https://github.com/ClickHouse/ClickHouse/pull/56342) ([Srikanth Chekuri](https://github.com/srikanthccv)).
* Allow to configure any kind of object storage with any kind of metadata type. [#58357](https://github.com/ClickHouse/ClickHouse/pull/58357) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Added `null_status_on_timeout_only_active` and `throw_only_active` modes for `distributed_ddl_output_mode` that allow to avoid waiting for inactive replicas. [#58350](https://github.com/ClickHouse/ClickHouse/pull/58350) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Allow partitions from tables with different partition expressions to be attached when the destination table partition expression doesn't re-partition/split the part. [#39507](https://github.com/ClickHouse/ClickHouse/pull/39507) ([Arthur Passos](https://github.com/arthurpassos)).
* Add function `arrayShingles` to compute subarrays, e.g. `arrayShingles([1, 2, 3, 4, 5], 3)` returns `[[1,2,3],[2,3,4],[3,4,5]]`. [#58396](https://github.com/ClickHouse/ClickHouse/pull/58396) ([Zheng Miao](https://github.com/zenmiao7)).
* Added functions `punycodeEncode`, `punycodeDecode`, `idnaEncode` and `idnaDecode` which are useful for translating international domain names to an ASCII representation according to the IDNA standard. [#58454](https://github.com/ClickHouse/ClickHouse/pull/58454) ([Robert Schulze](https://github.com/rschu1ze)).
* Added string similarity functions `dramerauLevenshteinDistance`, `jaroSimilarity` and `jaroWinklerSimilarity`. [#58531](https://github.com/ClickHouse/ClickHouse/pull/58531) ([Robert Schulze](https://github.com/rschu1ze)).

View File

@ -37,7 +37,7 @@ Keep an eye out for upcoming meetups around the world. Somewhere else you want u
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"
* **Recording available**: [**v23.10 Release Webinar**](https://www.youtube.com/watch?v=PGQS6uPb970) All the features of 23.10, one convenient video! Watch it now!
* **Recording available**: [**v24.1 Release Webinar**](https://www.youtube.com/watch?v=pBF9g0wGAGs) All the features of 24.1, one convenient video! Watch it now!
* **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU)

View File

@ -17,6 +17,7 @@ set (SRCS
getMemoryAmount.cpp
getPageSize.cpp
getThreadId.cpp
int8_to_string.cpp
JSON.cpp
mremap.cpp
phdr_cache.cpp

View File

@ -1,8 +1,11 @@
#include <stdexcept>
#include <fstream>
#include <base/getMemoryAmount.h>
#include <base/getPageSize.h>
#include <fstream>
#include <sstream>
#include <stdexcept>
#include <unistd.h>
#include <sys/types.h>
#include <sys/param.h>
@ -11,6 +14,80 @@
#endif
namespace
{
std::optional<uint64_t> getCgroupsV2MemoryLimit()
{
#if defined(OS_LINUX)
const std::filesystem::path default_cgroups_mount = "/sys/fs/cgroup";
/// This file exists iff the host has cgroups v2 enabled.
std::ifstream controllers_file(default_cgroups_mount / "cgroup.controllers");
if (!controllers_file.is_open())
return {};
/// Make sure that the memory controller is enabled.
/// - cgroup.controllers defines which controllers *can* be enabled.
/// - cgroup.subtree_control defines which controllers *are* enabled.
/// (see https://docs.kernel.org/admin-guide/cgroup-v2.html)
/// Caveat: nested groups may disable controllers. For simplicity, check only the top-level group.
/// ReadBufferFromFile subtree_control_file(default_cgroups_mount / "cgroup.subtree_control");
/// std::string subtree_control;
/// readString(subtree_control, subtree_control_file);
/// if (subtree_control.find("memory") == std::string::npos)
/// return {};
std::ifstream subtree_control_file(default_cgroups_mount / "cgroup.subtree_control");
std::stringstream subtree_control_buf;
subtree_control_buf << subtree_control_file.rdbuf();
std::string subtree_control = subtree_control_buf.str();
if (subtree_control.find("memory") == std::string::npos)
return {};
/// Identify the cgroup the process belongs to
/// All PIDs assigned to a cgroup are in /sys/fs/cgroups/{cgroup_name}/cgroup.procs
/// A simpler way to get the membership is:
std::ifstream cgroup_name_file("/proc/self/cgroup");
if (!cgroup_name_file.is_open())
return {};
std::stringstream cgroup_name_buf;
cgroup_name_buf << cgroup_name_file.rdbuf();
std::string cgroup_name = cgroup_name_buf.str();
if (!cgroup_name.empty() && cgroup_name.back() == '\n')
cgroup_name.pop_back(); /// remove trailing newline, if any
/// With cgroups v2, there will be a *single* line with prefix "0::/"
const std::string v2_prefix = "0::/";
if (!cgroup_name.starts_with(v2_prefix))
return {};
cgroup_name = cgroup_name.substr(v2_prefix.length());
std::filesystem::path current_cgroup = cgroup_name.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup_name);
/// Open the bottom-most nested memory limit setting file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.
while (current_cgroup != default_cgroups_mount.parent_path())
{
std::ifstream setting_file(current_cgroup / "memory.max");
if (setting_file.is_open())
{
uint64_t value;
if (setting_file >> value)
return {value};
else
return {}; /// e.g. the cgroups default "max"
}
current_cgroup = current_cgroup.parent_path();
}
return {};
#else
return {};
#endif
}
}
/** Returns the size of physical memory (RAM) in bytes.
* Returns 0 on unsupported platform
*/
@ -26,34 +103,27 @@ uint64_t getMemoryAmountOrZero()
uint64_t memory_amount = num_pages * page_size;
#if defined(OS_LINUX)
// Try to lookup at the Cgroup limit
// CGroups v2
std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max");
if (cgroupv2_limit.is_open())
{
uint64_t memory_limit = 0;
cgroupv2_limit >> memory_limit;
if (memory_limit > 0 && memory_limit < memory_amount)
memory_amount = memory_limit;
}
/// Respect the memory limit set by cgroups v2.
auto limit_v2 = getCgroupsV2MemoryLimit();
if (limit_v2.has_value() && *limit_v2 < memory_amount)
memory_amount = *limit_v2;
else
{
// CGroups v1
std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes");
if (cgroup_limit.is_open())
/// Cgroups v1 were replaced by v2 in 2015. The only reason we keep supporting v1 is that the transition to v2
/// has been slow. Caveat : Hierarchical groups as in v2 are not supported for v1, the location of the memory
/// limit (virtual) file is hard-coded.
/// TODO: check at the end of 2024 if we can get rid of v1.
std::ifstream limit_file_v1("/sys/fs/cgroup/memory/memory.limit_in_bytes");
if (limit_file_v1.is_open())
{
uint64_t memory_limit = 0; // in case of read error
cgroup_limit >> memory_limit;
if (memory_limit > 0 && memory_limit < memory_amount)
memory_amount = memory_limit;
uint64_t limit_v1;
if (limit_file_v1 >> limit_v1)
if (limit_v1 < memory_amount)
memory_amount = limit_v1;
}
}
#endif
return memory_amount;
}

View File

@ -0,0 +1,9 @@
#include <base/int8_to_string.h>
namespace std
{
std::string to_string(Int8 v) /// NOLINT (cert-dcl58-cpp)
{
return to_string(int8_t{v});
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <base/defines.h>
#include <base/types.h>
#include <fmt/format.h>
template <>
struct fmt::formatter<Int8> : fmt::formatter<int8_t>
{
};
namespace std
{
std::string to_string(Int8 v); /// NOLINT (cert-dcl58-cpp)
}

View File

@ -3,14 +3,29 @@
#include <cstdint>
#include <string>
/// This is needed for more strict aliasing. https://godbolt.org/z/xpJBSb https://stackoverflow.com/a/57453713
/// Using char8_t more strict aliasing (https://stackoverflow.com/a/57453713)
using UInt8 = char8_t;
/// Same for using signed _BitInt(8) (there isn't a signed char8_t, which would be more convenient)
/// See https://godbolt.org/z/fafnWEnnf
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wbit-int-extension"
using Int8 = signed _BitInt(8);
#pragma clang diagnostic pop
namespace std
{
template <>
struct hash<Int8> /// NOLINT (cert-dcl58-cpp)
{
size_t operator()(const Int8 x) const { return std::hash<int8_t>()(int8_t{x}); }
};
}
using UInt16 = uint16_t;
using UInt32 = uint32_t;
using UInt64 = uint64_t;
using Int8 = int8_t;
using Int16 = int16_t;
using Int32 = int32_t;
using Int64 = int64_t;

View File

@ -1,5 +1,5 @@
---
slug: /en/sql-reference/data-types/json
slug: /en/sql-reference/data-types/variant
sidebar_position: 55
sidebar_label: Variant
---

View File

@ -504,7 +504,7 @@ void Client::connect()
<< "It may lack support for new features." << std::endl
<< std::endl;
}
else if (client_version_tuple > server_version_tuple)
else if (client_version_tuple > server_version_tuple && server_display_name != "clickhouse-cloud")
{
std::cout << "ClickHouse server version is older than ClickHouse client. "
<< "It may indicate that the server is out of date and can be upgraded." << std::endl

View File

@ -234,6 +234,9 @@ public:
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
if (!this->data(rhs).value.size())
return;
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);

View File

@ -2766,7 +2766,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
{
if (identifier_lookup.isExpressionLookup())
{
return tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, 1 /*identifier_bind_size*/, it->second, {}, scope);
return tryResolveIdentifierFromCompoundExpression(
identifier_lookup.identifier,
1 /*identifier_bind_size*/,
it->second,
{} /* compound_expression_source */,
scope,
identifier_resolve_settings.allow_to_check_join_tree /* can_be_not_found */);
}
else if (identifier_lookup.isFunctionLookup() || identifier_lookup.isTableExpressionLookup())
{

View File

@ -14,12 +14,13 @@ enum Time
Seconds,
};
template <Time time>
template <Time unit>
struct ProfileEventTimeIncrement
{
explicit ProfileEventTimeIncrement<time>(ProfileEvents::Event event_)
: event(event_), watch(CLOCK_MONOTONIC) {}
template <Time time = unit>
UInt64 elapsed()
{
if constexpr (time == Time::Nanoseconds)

View File

@ -9,10 +9,11 @@
#include <base/defines.h>
#include <base/errnoToString.h>
#include <base/int8_to_string.h>
#include <base/scope_guard.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/Logger.h>
#include <Common/AtomicLogger.h>
#include <Common/Logger.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/StackTrace.h>
#include <fmt/format.h>

View File

@ -2,6 +2,7 @@
#include <atomic>
#include <memory>
#include <mutex>
#include <base/defines.h>
@ -20,6 +21,9 @@
* } // now we finish own current version; if the version is outdated and no one else is using it - it will be destroyed.
*
* All methods are thread-safe.
*
* Standard library does not have atomic_shared_ptr, and we do not use std::atomic* operations on shared_ptr,
* because standard library implementation uses fixed table of mutexes, and it is better to avoid contention here.
*/
template <typename T>
class MultiVersion
@ -42,25 +46,37 @@ public:
MultiVersion & operator=(MultiVersion && src)
{
if (this != &src)
std::atomic_store(&current_version, std::atomic_exchange(&src.current_version, Version{}));
{
Version version;
{
std::lock_guard<std::mutex> lock(src.mutex);
src.current_version.swap(version);
}
std::lock_guard<std::mutex> lock(mutex);
current_version = std::move(version);
}
return *this;
}
/// Obtain current version for read-only usage. Returns shared_ptr, that manages lifetime of version.
Version get() const
{
return std::atomic_load(&current_version);
std::lock_guard<std::mutex> lock(mutex);
return current_version;
}
/// TODO: replace atomic_load/store() on shared_ptr (which is deprecated as of C++20) by C++20 std::atomic<std::shared_ptr>.
/// Clang 15 currently does not support it.
/// Update an object with new version.
void set(std::unique_ptr<const T> && value)
{
std::atomic_store(&current_version, Version{std::move(value)});
Version version{std::move(value)};
std::lock_guard<std::mutex> lock(mutex);
current_version = std::move(version);
}
private:
mutable std::mutex mutex;
Version current_version;
};

View File

@ -197,6 +197,15 @@
M(MergeTreeDataWriterBlocks, "Number of blocks INSERTed to MergeTree tables. Each block forms a data part of level zero.") \
M(MergeTreeDataWriterBlocksAlreadySorted, "Number of blocks INSERTed to MergeTree tables that appeared to be already sorted.") \
\
M(MergeTreeDataWriterSkipIndicesCalculationMicroseconds, "Time spent calculating skip indices") \
M(MergeTreeDataWriterStatisticsCalculationMicroseconds, "Time spent calculating statistics") \
M(MergeTreeDataWriterSortingBlocksMicroseconds, "Time spent sorting blocks") \
M(MergeTreeDataWriterMergingBlocksMicroseconds, "Time spent merging input blocks (for special MergeTree engines)") \
M(MergeTreeDataWriterProjectionsCalculationMicroseconds, "Time spent calculating projections") \
M(MergeTreeDataProjectionWriterSortingBlocksMicroseconds, "Time spent sorting blocks (for projection it might be a key different from table's sorting key)") \
M(MergeTreeDataProjectionWriterMergingBlocksMicroseconds, "Time spent merging blocks") \
M(MutateTaskProjectionsCalculationMicroseconds, "Time spent calculating projections") \
\
M(InsertedWideParts, "Number of parts inserted in Wide format.") \
M(InsertedCompactParts, "Number of parts inserted in Compact format.") \
M(MergedIntoWideParts, "Number of parts merged into Wide format.") \

View File

@ -206,6 +206,9 @@ public:
*/
void merge(const Self & rhs)
{
if (!rhs.size())
return;
UInt64 m1 = 0;
UInt64 m2 = 0;

View File

@ -1,7 +1,6 @@
#include "getNumberOfPhysicalCPUCores.h"
#include <filesystem>
#include "config.h"
#if defined(OS_LINUX)
# include <cmath>
# include <fstream>
@ -34,9 +33,9 @@ int32_t readFrom(const std::filesystem::path & filename, int default_value)
uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
{
uint32_t quota_count = default_cpu_count;
std::filesystem::path prefix = "/sys/fs/cgroup";
std::filesystem::path default_cgroups_mount = "/sys/fs/cgroup";
/// cgroupsv2
std::ifstream contr_file(prefix / "cgroup.controllers");
std::ifstream contr_file(default_cgroups_mount / "cgroup.controllers");
if (contr_file.is_open())
{
/// First, we identify the cgroup the process belongs
@ -51,16 +50,15 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
std::filesystem::path current_cgroup;
if (cgroup_name.empty())
current_cgroup = prefix;
current_cgroup = default_cgroups_mount;
else
current_cgroup = prefix / cgroup_name;
current_cgroup = default_cgroups_mount / cgroup_name;
// Looking for cpu.max in directories from the current cgroup to the top level
// It does not stop on the first time since the child could have a greater value than parent
while (current_cgroup != prefix.parent_path())
while (current_cgroup != default_cgroups_mount.parent_path())
{
std::ifstream cpu_max_file(current_cgroup / "cpu.max");
current_cgroup = current_cgroup.parent_path();
if (cpu_max_file.is_open())
{
std::string cpu_limit_str;
@ -72,10 +70,11 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
quota_count = std::min(static_cast<uint32_t>(ceil(cpu_limit / cpu_period)), quota_count);
}
}
current_cgroup = current_cgroup.parent_path();
}
current_cgroup = prefix / cgroup_name;
current_cgroup = default_cgroups_mount / cgroup_name;
// Looking for cpuset.cpus.effective in directories from the current cgroup to the top level
while (current_cgroup != prefix.parent_path())
while (current_cgroup != default_cgroups_mount.parent_path())
{
std::ifstream cpuset_cpus_file(current_cgroup / "cpuset.cpus.effective");
current_cgroup = current_cgroup.parent_path();
@ -113,8 +112,8 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
/// cgroupsv1
/// Return the number of milliseconds per period process is guaranteed to run.
/// -1 for no quota
int cgroup_quota = readFrom(prefix / "cpu/cpu.cfs_quota_us", -1);
int cgroup_period = readFrom(prefix / "cpu/cpu.cfs_period_us", -1);
int cgroup_quota = readFrom(default_cgroups_mount / "cpu/cpu.cfs_quota_us", -1);
int cgroup_period = readFrom(default_cgroups_mount / "cpu/cpu.cfs_period_us", -1);
if (cgroup_quota > -1 && cgroup_period > 0)
quota_count = static_cast<uint32_t>(ceil(static_cast<float>(cgroup_quota) / static_cast<float>(cgroup_period)));
@ -178,24 +177,25 @@ catch (...)
unsigned getNumberOfPhysicalCPUCoresImpl()
{
unsigned cpu_count = std::thread::hardware_concurrency(); /// logical cores (with SMT/HyperThreading)
unsigned cores = std::thread::hardware_concurrency(); /// logical cores (with SMT/HyperThreading)
#if defined(__x86_64__) && defined(OS_LINUX)
/// Most x86_64 CPUs have 2-way SMT (Hyper-Threading).
/// Aarch64 and RISC-V don't have SMT so far.
/// POWER has SMT and it can be multi-way (e.g. 8-way), but we don't know how ClickHouse really behaves, so use all of them.
#if defined(__x86_64__) && defined(OS_LINUX)
///
/// On really big machines, SMT is detrimental to performance (+ ~5% overhead in ClickBench). On such machines, we limit ourself to the physical cores.
/// Few cores indicate it is a small machine, runs in a VM or is a limited cloud instance --> it is reasonable to use all the cores.
if (cpu_count >= 32)
cpu_count = physical_concurrency();
if (cores >= 32)
cores = physical_concurrency();
#endif
#if defined(OS_LINUX)
cpu_count = getCGroupLimitedCPUCores(cpu_count);
cores = getCGroupLimitedCPUCores(cores);
#endif
return cpu_count;
return cores;
}
}
@ -203,6 +203,6 @@ unsigned getNumberOfPhysicalCPUCoresImpl()
unsigned getNumberOfPhysicalCPUCores()
{
/// Calculate once.
static auto res = getNumberOfPhysicalCPUCoresImpl();
return res;
static auto cores = getNumberOfPhysicalCPUCoresImpl();
return cores;
}

View File

@ -216,9 +216,8 @@ using NearestFieldType = typename NearestFieldTypeImpl<T>::Type;
template <> struct NearestFieldTypeImpl<char> { using Type = std::conditional_t<is_signed_v<char>, Int64, UInt64>; };
template <> struct NearestFieldTypeImpl<signed char> { using Type = Int64; };
template <> struct NearestFieldTypeImpl<unsigned char> { using Type = UInt64; };
#ifdef __cpp_char8_t
template <> struct NearestFieldTypeImpl<char8_t> { using Type = UInt64; };
#endif
template <> struct NearestFieldTypeImpl<Int8> { using Type = Int64; };
template <> struct NearestFieldTypeImpl<UInt16> { using Type = UInt64; };
template <> struct NearestFieldTypeImpl<UInt32> { using Type = UInt64; };

View File

@ -562,7 +562,7 @@ class IColumn;
M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
M(DefaultTableEngine, default_temporary_table_engine, DefaultTableEngine::Memory, "Default table engine used when ENGINE is not set in CREATE TEMPORARY statement.",0) \
M(DefaultTableEngine, default_table_engine, DefaultTableEngine::None, "Default table engine used when ENGINE is not set in CREATE statement.",0) \
M(DefaultTableEngine, default_table_engine, DefaultTableEngine::MergeTree, "Default table engine used when ENGINE is not set in CREATE statement.",0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
@ -751,7 +751,7 @@ class IColumn;
M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \
M(Bool, wait_for_async_insert, true, "If true wait for processing of asynchronous insertion", 0) \
M(Seconds, wait_for_async_insert_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "Timeout for waiting for processing asynchronous insertion", 0) \
M(UInt64, async_insert_max_data_size, 1000000, "Maximum size in bytes of unparsed data collected per query before being inserted", 0) \
M(UInt64, async_insert_max_data_size, 10485760, "Maximum size in bytes of unparsed data collected per query before being inserted", 0) \
M(UInt64, async_insert_max_query_number, 450, "Maximum number of insert queries before being inserted", 0) \
M(Milliseconds, async_insert_poll_timeout_ms, 10, "Timeout for polling data from asynchronous insert queue", 0) \
M(Bool, async_insert_use_adaptive_busy_timeout, true, "If it is set to true, use adaptive busy timeout for asynchronous inserts", 0) \

View File

@ -84,7 +84,8 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.2", {{"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"},
{"24.2", {{"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."},
{"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"},
{"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"},
{"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"},
{"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"},
@ -111,6 +112,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"},
{"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."},
{"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."},
{"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"},
{"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."},

View File

@ -670,8 +670,7 @@ struct ArrayElementStringImpl
ColumnArray::Offset current_offset = 0;
/// get the total result bytes at first, and reduce the cost of result_data.resize.
size_t total_result_bytes = 0;
ColumnString::Chars zero_buf(1);
zero_buf.push_back(0);
ColumnString::Chars zero_buf(16, '\0'); /// Needs 15 extra bytes for memcpySmallAllowReadWriteOverflow15
std::vector<std::pair<const ColumnString::Char *, UInt64>> selected_bufs;
selected_bufs.reserve(size);
for (size_t i = 0; i < size; ++i)
@ -737,8 +736,7 @@ struct ArrayElementStringImpl
size_t size = offsets.size();
result_offsets.resize(size);
ColumnString::Chars zero_buf(1);
zero_buf.push_back(0);
ColumnString::Chars zero_buf(16, '\0'); /// Needs 15 extra bytes for memcpySmallAllowReadWriteOverflow15
ColumnArray::Offset current_offset = 0;
/// get the total result bytes at first, and reduce the cost of result_data.resize.
size_t total_result_bytes = 0;

View File

@ -49,9 +49,9 @@ template void divideImpl<uint32_t, char8_t, uint32_t>(const uint32_t * __restric
template void divideImpl<int64_t, int64_t, int64_t>(const int64_t * __restrict, int64_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int32_t, int64_t>(const int64_t * __restrict, int32_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int16_t, int64_t>(const int64_t * __restrict, int16_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int8_t, int64_t>(const int64_t * __restrict, int8_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, Int8, int64_t>(const int64_t * __restrict, Int8, int64_t * __restrict, size_t);
template void divideImpl<int32_t, int64_t, int32_t>(const int32_t * __restrict, int64_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int32_t, int32_t>(const int32_t * __restrict, int32_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int16_t, int32_t>(const int32_t * __restrict, int16_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int8_t, int32_t>(const int32_t * __restrict, int8_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, Int8, int32_t>(const int32_t * __restrict, Int8, int32_t * __restrict, size_t);

View File

@ -12,6 +12,10 @@
#include <libdivide.h>
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wbit-int-extension"
using Int8 = signed _BitInt(8);
#pragma clang diagnostic pop
namespace NAMESPACE
{
@ -62,11 +66,11 @@ template void divideImpl<uint32_t, char8_t, uint32_t>(const uint32_t * __restric
template void divideImpl<int64_t, int64_t, int64_t>(const int64_t * __restrict, int64_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int32_t, int64_t>(const int64_t * __restrict, int32_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int16_t, int64_t>(const int64_t * __restrict, int16_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, int8_t, int64_t>(const int64_t * __restrict, int8_t, int64_t * __restrict, size_t);
template void divideImpl<int64_t, Int8, int64_t>(const int64_t * __restrict, Int8, int64_t * __restrict, size_t);
template void divideImpl<int32_t, int64_t, int32_t>(const int32_t * __restrict, int64_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int32_t, int32_t>(const int32_t * __restrict, int32_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int16_t, int32_t>(const int32_t * __restrict, int16_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, int8_t, int32_t>(const int32_t * __restrict, int8_t, int32_t * __restrict, size_t);
template void divideImpl<int32_t, Int8, int32_t>(const int32_t * __restrict, Int8, int32_t * __restrict, size_t);
}

View File

@ -230,13 +230,8 @@ inline void fillConstantConstant(const ArrayCond & cond, A a, B b, ArrayResult &
{
size_t size = cond.size();
/// Int8(alias type of uint8_t) has special aliasing properties that prevents compiler from auto-vectorizing for below codes, refer to https://gist.github.com/alexei-zaripov/dcc14c78819c5f1354afe8b70932007c
///
/// for (size_t i = 0; i < size; ++i)
/// res[i] = cond[i] ? static_cast<Int8>(a) : static_cast<Int8>(b);
///
/// Therefore, we manually optimize it by avoiding branch miss when ResultType is Int8. Other types like (U)Int128|256 or Decimal128/256 also benefit from this optimization.
if constexpr (std::is_same_v<ResultType, Int8> || is_over_big_int<ResultType>)
/// We manually optimize the loop for types like (U)Int128|256 or Decimal128/256 to avoid branches
if constexpr (is_over_big_int<ResultType>)
{
alignas(64) const ResultType ab[2] = {static_cast<ResultType>(a), static_cast<ResultType>(b)};
for (size_t i = 0; i < size; ++i)

View File

@ -1,14 +1,14 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <Core/ColumnNumbers.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVariant.h>
#include <Core/ColumnNumbers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Functions/PerformanceAdaptors.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace
@ -21,10 +21,7 @@ class FunctionIsNotNull : public IFunction
public:
static constexpr auto name = "isNotNull";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionIsNotNull>();
}
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIsNotNull>(); }
std::string getName() const override
{
@ -52,9 +49,9 @@ public:
const auto & discriminators = checkAndGetColumn<ColumnVariant>(*elem.column)->getLocalDiscriminators();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.reserve(discriminators.size());
for (auto discr : discriminators)
data.push_back(discr != ColumnVariant::NULL_DISCRIMINATOR);
data.resize(discriminators.size());
for (size_t i = 0; i < discriminators.size(); ++i)
data[i] = discriminators[i] != ColumnVariant::NULL_DISCRIMINATOR;
return res;
}
@ -64,9 +61,9 @@ public:
const size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.reserve(low_cardinality_column->size());
data.resize(low_cardinality_column->size());
for (size_t i = 0; i != low_cardinality_column->size(); ++i)
data.push_back(low_cardinality_column->getIndexAt(i) != null_index);
data[i] = (low_cardinality_column->getIndexAt(i) != null_index);
return res;
}
@ -76,10 +73,7 @@ public:
auto res_column = ColumnUInt8::create(input_rows_count);
const auto & src_data = nullable->getNullMapData();
auto & res_data = assert_cast<ColumnUInt8 &>(*res_column).getData();
for (size_t i = 0; i < input_rows_count; ++i)
res_data[i] = !src_data[i];
vector(src_data, res_data);
return res_column;
}
else
@ -88,8 +82,34 @@ public:
return DataTypeUInt8().createColumnConst(elem.column->size(), 1u);
}
}
};
private:
MULTITARGET_FUNCTION_AVX2_SSE42(
MULTITARGET_FUNCTION_HEADER(static void NO_INLINE), vectorImpl, MULTITARGET_FUNCTION_BODY((const PaddedPODArray<UInt8> & null_map, PaddedPODArray<UInt8> & res) /// NOLINT
{
size_t size = null_map.size();
for (size_t i = 0; i < size; ++i)
res[i] = !null_map[i];
}))
static void NO_INLINE vector(const PaddedPODArray<UInt8> & null_map, PaddedPODArray<UInt8> & res)
{
#if USE_MULTITARGET_CODE
if (isArchSupported(TargetArch::AVX2))
{
vectorImplAVX2(null_map, res);
return;
}
if (isArchSupported(TargetArch::SSE42))
{
vectorImplSSE42(null_map, res);
return;
}
#endif
vectorImpl(null_map, res);
}
};
}
REGISTER_FUNCTION(IsNotNull)

View File

@ -100,7 +100,10 @@ void RewriteSumFunctionWithSumAndCountMatcher::visit(const ASTFunction & functio
if (!new_ast)
return;
else
{
new_ast->setAlias(ast->tryGetAlias());
ast = new_ast;
}
}
else if (column_id == 1)
{
@ -116,7 +119,10 @@ void RewriteSumFunctionWithSumAndCountMatcher::visit(const ASTFunction & functio
if (!new_ast)
return;
else
{
new_ast->setAlias(ast->tryGetAlias());
ast = new_ast;
}
}
}

View File

@ -130,6 +130,11 @@ namespace DB
reinterpret_cast<const uint8_t *>(internal_data.data() + start),
end - start,
reinterpret_cast<const uint8_t *>(arrow_null_bytemap_raw_ptr));
else if constexpr (std::is_same_v<NumericType, Int8>)
status = builder.AppendValues(
reinterpret_cast<const int8_t *>(internal_data.data() + start),
end - start,
reinterpret_cast<const uint8_t *>(arrow_null_bytemap_raw_ptr));
else
status = builder.AppendValues(internal_data.data() + start, end - start, reinterpret_cast<const uint8_t *>(arrow_null_bytemap_raw_ptr));
checkStatus(status, write_column->getName(), format_name);

View File

@ -419,7 +419,11 @@ namespace
void read(GRPCQueryInfo & query_info_, const CompletionCallback & callback) override
{
if (!query_info.has_value())
{
callback(false);
return;
}
query_info_ = std::move(query_info).value();
query_info.reset();
callback(true);
@ -486,7 +490,11 @@ namespace
void read(GRPCQueryInfo & query_info_, const CompletionCallback & callback) override
{
if (!query_info.has_value())
{
callback(false);
return;
}
query_info_ = std::move(query_info).value();
query_info.reset();
callback(true);

View File

@ -1,6 +1,14 @@
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
#include <Common/ElapsedTimeProfileEventIncrement.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/logger_useful.h>
namespace ProfileEvents
{
extern const Event MergeTreeDataWriterSkipIndicesCalculationMicroseconds;
extern const Event MergeTreeDataWriterStatisticsCalculationMicroseconds;
}
namespace DB
{
@ -148,6 +156,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
, default_codec(default_codec_)
, compute_granularity(index_granularity.empty())
, compress_primary_key(settings.compress_primary_key)
, execution_stats(skip_indices.size(), stats.size())
, log(getLogger(storage.getLogName() + " (DataPartWriter)"))
{
if (settings.blocks_are_granules_size && !index_granularity.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
@ -329,9 +339,12 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc
void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block & block)
{
for (const auto & stat_ptr : stats)
for (size_t i = 0; i < stats.size(); ++i)
{
const auto & stat_ptr = stats[i];
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataWriterStatisticsCalculationMicroseconds);
stat_ptr->update(block.getByName(stat_ptr->columnName()).column);
execution_stats.statistics_build_us[i] += watch.elapsed();
}
}
@ -378,10 +391,14 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
writeBinaryLittleEndian(1UL, marks_out);
}
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataWriterSkipIndicesCalculationMicroseconds);
size_t pos = granule.start_row;
skip_indices_aggregators[i]->update(skip_indexes_block, &pos, granule.rows_to_write);
if (granule.is_complete)
++skip_index_accumulated_marks[i];
execution_stats.skip_indices_build_us[i] += watch.elapsed();
}
}
}
@ -481,6 +498,9 @@ void MergeTreeDataPartWriterOnDisk::finishStatisticsSerialization(bool sync)
if (sync)
stream->sync();
}
for (size_t i = 0; i < stats.size(); ++i)
LOG_DEBUG(log, "Spent {} ms calculating statistics {} for the part {}", execution_stats.statistics_build_us[i] / 1000, stats[i]->columnName(), data_part->name);
}
void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums)
@ -504,6 +524,10 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
}
for (auto & store: gin_index_stores)
store.second->finalize();
for (size_t i = 0; i < skip_indices.size(); ++i)
LOG_DEBUG(log, "Spent {} ms calculating index {} for the part {}", execution_stats.skip_indices_build_us[i] / 1000, skip_indices[i]->index.name, data_part->name);
gin_index_stores.clear();
skip_indices_streams.clear();
skip_indices_aggregators.clear();

View File

@ -190,6 +190,20 @@ private:
void initStatistics();
virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) = 0;
struct ExecutionStatistics
{
ExecutionStatistics(size_t skip_indices_cnt, size_t stats_cnt)
: skip_indices_build_us(skip_indices_cnt, 0), statistics_build_us(stats_cnt, 0)
{
}
std::vector<size_t> skip_indices_build_us; // [i] corresponds to the i-th index
std::vector<size_t> statistics_build_us; // [i] corresponds to the i-th stat
};
ExecutionStatistics execution_stats;
LoggerPtr log;
};
}

View File

@ -1,21 +1,22 @@
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Columns/ColumnConst.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/HashTable/HashMap.h>
#include <Common/Exception.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/ObjectUtils.h>
#include <Disks/createVolume.h>
#include <IO/HashingWriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/AggregationCommon.h>
#include <Interpreters/Context.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <IO/HashingWriteBuffer.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/ObjectUtils.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
#include <Processors/TTL/ITTLAlgorithm.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Common/ElapsedTimeProfileEventIncrement.h>
#include <Common/Exception.h>
#include <Common/HashTable/HashMap.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/typeid_cast.h>
#include <Parsers/queryToString.h>
@ -35,11 +36,16 @@ namespace ProfileEvents
extern const Event MergeTreeDataWriterRows;
extern const Event MergeTreeDataWriterUncompressedBytes;
extern const Event MergeTreeDataWriterCompressedBytes;
extern const Event MergeTreeDataWriterSortingBlocksMicroseconds;
extern const Event MergeTreeDataWriterMergingBlocksMicroseconds;
extern const Event MergeTreeDataWriterProjectionsCalculationMicroseconds;
extern const Event MergeTreeDataProjectionWriterBlocks;
extern const Event MergeTreeDataProjectionWriterBlocksAlreadySorted;
extern const Event MergeTreeDataProjectionWriterRows;
extern const Event MergeTreeDataProjectionWriterUncompressedBytes;
extern const Event MergeTreeDataProjectionWriterCompressedBytes;
extern const Event MergeTreeDataProjectionWriterSortingBlocksMicroseconds;
extern const Event MergeTreeDataProjectionWriterMergingBlocksMicroseconds;
extern const Event RejectedInserts;
}
@ -472,6 +478,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
IColumn::Permutation perm;
if (!sort_description.empty())
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataWriterSortingBlocksMicroseconds);
if (!isAlreadySorted(block, sort_description))
{
stableGetPermutation(block, sort_description, perm);
@ -483,7 +491,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names;
if (context->getSettingsRef().optimize_on_insert)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataWriterMergingBlocksMicroseconds);
block = mergeBlock(block, sort_description, partition_key_columns, perm_ptr, data.merging_params);
}
/// Size of part would not be greater than block.bytes() + epsilon
size_t expected_size = block.bytes();
@ -588,7 +599,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
for (const auto & projection : metadata_snapshot->getProjections())
{
auto projection_block = projection.calculate(block, context);
Block projection_block;
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataWriterProjectionsCalculationMicroseconds);
projection_block = projection.calculate(block, context);
LOG_DEBUG(log, "Spent {} ms calculating projection {} for the part {}", watch.elapsed() / 1000, projection.name, new_data_part->name);
}
if (projection_block.rows())
{
auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get());
@ -685,6 +702,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
IColumn::Permutation perm;
if (!sort_description.empty())
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataProjectionWriterSortingBlocksMicroseconds);
if (!isAlreadySorted(block, sort_description))
{
stableGetPermutation(block, sort_description, perm);
@ -696,6 +715,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
if (projection.type == ProjectionDescription::Type::Aggregate)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataProjectionWriterMergingBlocksMicroseconds);
MergeTreeData::MergingParams projection_merging_params;
projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating;
block = mergeBlock(block, sort_description, {}, perm_ptr, projection_merging_params);

View File

@ -28,6 +28,11 @@
#include <Common/ProfileEventsScope.h>
namespace ProfileEvents
{
extern const Event MutateTaskProjectionsCalculationMicroseconds;
}
namespace CurrentMetrics
{
extern const Metric PartMutation;
@ -1242,7 +1247,13 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections()
for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i)
{
const auto & projection = *ctx->projections_to_build[i];
auto projection_block = projection_squashes[i].add(projection.calculate(cur_block, ctx->context));
Block projection_block;
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds);
projection_block = projection_squashes[i].add(projection.calculate(cur_block, ctx->context));
}
if (projection_block)
{
auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart(

View File

@ -313,7 +313,7 @@ void MaterializedPostgreSQLConsumer::readTupleData(
Int32 col_len = readInt32(message, pos, size);
String value;
for (Int32 i = 0; i < col_len; ++i)
value += readInt8(message, pos, size);
value += static_cast<char>(readInt8(message, pos, size));
insertValue(storage_data, value, column_idx);
break;

View File

@ -1533,10 +1533,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster(
IColumn::Selector StorageDistributed::createSelector(const ClusterPtr cluster, const ColumnWithTypeAndName & result)
{
const auto & slot_to_shard = cluster->getSlotToShard();
const IColumn * column = result.column.get();
if (const auto * col_const = typeid_cast<const ColumnConst *>(column))
column = &col_const->getDataColumn();
// If result.type is DataTypeLowCardinality, do shard according to its dictionaryType
#define CREATE_FOR_TYPE(TYPE) \

View File

@ -180,7 +180,7 @@ using Paths = std::deque<std::pair<String, ZkPathType>>;
class ReadFromSystemZooKeeper final : public SourceStepWithFilter
{
public:
ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info_, ContextPtr context_);
ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info_, UInt64 max_block_size_, ContextPtr context_);
String getName() const override { return "ReadFromSystemZooKeeper"; }
@ -189,13 +189,41 @@ public:
void applyFilters() override;
private:
void fillData(MutableColumns & res_columns);
std::shared_ptr<const StorageLimitsList> storage_limits;
const UInt64 max_block_size;
ContextPtr context;
Paths paths;
};
class SystemZooKeeperSource : public ISource
{
public:
SystemZooKeeperSource(
Paths && paths_,
Block header_,
UInt64 max_block_size_,
ContextPtr context_)
: ISource(header_)
, max_block_size(max_block_size_)
, paths(std::move(paths_))
, context(std::move(context_))
{
}
String getName() const override { return "SystemZooKeeper"; }
protected:
Chunk generate() override;
private:
const UInt64 max_block_size;
Paths paths;
ContextPtr context;
bool started = false;
};
StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_)
: IStorage(table_id_)
{
@ -211,11 +239,11 @@ void StorageSystemZooKeeper::read(
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t /*max_block_size*/,
size_t max_block_size,
size_t /*num_streams*/)
{
auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals());
auto read_step = std::make_unique<ReadFromSystemZooKeeper>(header, query_info, context);
auto read_step = std::make_unique<ReadFromSystemZooKeeper>(header, query_info, max_block_size, context);
query_plan.addStep(std::move(read_step));
}
@ -414,7 +442,7 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont
for (const auto * node : filter_nodes)
extractPathImpl(*node, res, context, allow_unrestricted);
if (filter_nodes.empty() && allow_unrestricted)
if (res.empty() && allow_unrestricted)
res.emplace_back("/", ZkPathType::Recurse);
return res;
@ -426,8 +454,26 @@ void ReadFromSystemZooKeeper::applyFilters()
paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper);
}
void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
Chunk SystemZooKeeperSource::generate()
{
if (paths.empty())
{
if (!started)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"SELECT from system.zookeeper table must contain condition like path = 'path' "
"or path IN ('path1','path2'...) or path IN (subquery) "
"in WHERE clause unless `set allow_unrestricted_reads_from_keeper = 'true'`.");
/// No more work
return {};
}
started = true;
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
size_t row_count = 0;
QueryStatusPtr query_status = context->getProcessListElement();
const auto & settings = context->getSettingsRef();
@ -453,12 +499,6 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
return zookeeper;
};
if (paths.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"SELECT from system.zookeeper table must contain condition like path = 'path' "
"or path IN ('path1','path2'...) or path IN (subquery) "
"in WHERE clause unless `set allow_unrestricted_reads_from_keeper = 'true'`.");
const Int64 max_inflight_requests = std::max<Int64>(1, context->getSettingsRef().max_download_threads.value);
struct ListTask
@ -476,6 +516,16 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
if (query_status)
query_status->checkTimeLimit();
/// Check if the block is big enough already
if (max_block_size > 0 && row_count > 0)
{
size_t total_size = 0;
for (const auto & column : res_columns)
total_size += column->byteSize();
if (total_size > max_block_size)
break;
}
list_tasks.clear();
std::vector<String> paths_to_list;
while (!paths.empty() && static_cast<Int64>(list_tasks.size()) < max_inflight_requests)
@ -519,8 +569,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
continue;
auto & task = list_tasks[list_task_idx];
if (auto elem = context->getProcessListElement())
elem->checkTimeLimit();
if (query_status)
query_status->checkTimeLimit();
Strings nodes = std::move(list_result.names);
@ -557,8 +607,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
auto & get_task = get_tasks[i];
auto & list_task = list_tasks[get_task.list_task_idx];
if (auto elem = context->getProcessListElement())
elem->checkTimeLimit();
if (query_status)
query_status->checkTimeLimit();
// Deduplication
String key = list_task.path_part + '/' + get_task.node;
@ -584,17 +634,22 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
res_columns[col_num++]->insert(
list_task.path); /// This is the original path. In order to process the request, condition in WHERE should be triggered.
++row_count;
if (list_task.path_type != ZkPathType::Exact && res.stat.numChildren > 0)
{
paths.emplace_back(key, ZkPathType::Recurse);
}
}
}
return Chunk(std::move(res_columns), row_count);
}
ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, ContextPtr context_)
ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, UInt64 max_block_size_, ContextPtr context_)
: SourceStepWithFilter({.header = header})
, storage_limits(query_info.storage_limits)
, max_block_size(max_block_size_)
, context(std::move(context_))
{
}
@ -602,13 +657,7 @@ ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQue
void ReadFromSystemZooKeeper::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
const auto & header = getOutputStream().header;
MutableColumns res_columns = header.cloneEmptyColumns();
fillData(res_columns);
UInt64 num_rows = res_columns.at(0)->size();
Chunk chunk(std::move(res_columns), num_rows);
auto source = std::make_shared<SourceFromSingleChunk>(header, std::move(chunk));
auto source = std::make_shared<SystemZooKeeperSource>(std::move(paths), header, max_block_size, context);
source->setStorageLimits(storage_limits);
processors.emplace_back(source);
pipeline.init(Pipe(std::move(source)));

View File

@ -23,7 +23,6 @@
02428_parameterized_view
02493_inconsistent_hex_and_binary_number
02575_merge_prewhere_different_default_kind
00917_multiple_joins_denny_crane
02725_agg_projection_resprect_PK
02763_row_policy_storage_merge_alias
02818_parameterized_view_with_cte_multiple_usage

View File

@ -10,14 +10,17 @@ from pathlib import Path
from shutil import copy2
from typing import List, Optional, Union
# isort: off
from github.Commit import Commit
# isort: on
from build_download_helper import download_build_with_progress
from commit_status_helper import post_commit_status
from compress_files import SUFFIX, compress_fast, decompress_fast
from env_helper import CI, RUNNER_TEMP, S3_BUILDS_BUCKET
from git_helper import SHA_REGEXP
from report import HEAD_HTML_TEMPLATE, FOOTER_HTML_TEMPLATE
from report import FOOTER_HTML_TEMPLATE, HEAD_HTML_TEMPLATE, SUCCESS
from s3_helper import S3Helper
ARTIFACTS_PATH = Path(RUNNER_TEMP) / "artifacts"
@ -128,9 +131,7 @@ class ArtifactsHelper:
@staticmethod
def post_commit_status(commit: Commit, url: str) -> None:
post_commit_status(
commit, "success", url, "Artifacts for workflow", "Artifacts"
)
post_commit_status(commit, SUCCESS, url, "Artifacts for workflow", "Artifacts")
def _regenerate_index(self) -> None:
if CI:

View File

@ -7,13 +7,11 @@ import sys
from pathlib import Path
from build_download_helper import get_build_name_for_check, read_build_urls
from clickhouse_helper import (
CiLogsCredentials,
)
from clickhouse_helper import CiLogsCredentials
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPORT_PATH, TEMP_PATH
from pr_info import PRInfo
from report import JobReport
from report import FAIL, FAILURE, OK, SUCCESS, JobReport, TestResult
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -144,12 +142,16 @@ def main():
with open(workspace_path / "description.txt", "r", encoding="utf-8") as desc_f:
description = desc_f.readline().rstrip("\n")
except:
status = "failure"
status = FAILURE
description = "Task failed: $?=" + str(retcode)
test_result = TestResult(description, OK)
if "fail" in status:
test_result.status = FAIL
JobReport(
description=description,
test_results=[],
test_results=[test_result],
status=status,
start_time=stopwatch.start_time_str,
duration=stopwatch.duration_seconds,
@ -158,7 +160,7 @@ def main():
).dump()
logging.info("Result: '%s', '%s'", status, description)
if status == "failure":
if status != SUCCESS:
sys.exit(1)

View File

@ -1,17 +1,20 @@
#!/usr/bin/env python3
from pathlib import Path
from typing import List, Tuple, Optional
import argparse
import csv
import logging
from pathlib import Path
from typing import List, Optional, Tuple
# isort: off
from github import Github
# isort: on
from commit_status_helper import get_commit, post_commit_status
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResults, TestResult
from report import ERROR, SUCCESS, TestResult, TestResults
from s3_helper import S3Helper
from upload_result_helper import upload_results
@ -49,7 +52,7 @@ def process_result(file_path: Path) -> Tuple[bool, TestResults, Optional[str]]:
)
return False, [TestResult(f"{prefix}: {description}", status)], "Check failed"
is_ok = state == "success"
is_ok = state == SUCCESS
if is_ok and report_url == "null":
return is_ok, test_results, None
@ -111,7 +114,7 @@ def main():
commit = get_commit(gh, pr_info.sha)
post_commit_status(
commit,
"success" if is_ok else "error",
SUCCESS if is_ok else ERROR,
report_url,
description,
check_name_with_group,

View File

@ -1,21 +1,21 @@
import argparse
import concurrent.futures
from dataclasses import asdict, dataclass
from enum import Enum
import json
import logging
import os
import re
import subprocess
import sys
from pathlib import Path
import time
from dataclasses import asdict, dataclass
from enum import Enum
from pathlib import Path
from typing import Any, Dict, List, Optional, Sequence, Union
import docker_images_helper
import upload_result_helper
from build_check import get_release_or_pr
from ci_config import CI_CONFIG, Build, Labels, JobNames
from ci_config import CI_CONFIG, Build, JobNames, Labels
from ci_utils import GHActions, is_hex
from clickhouse_helper import (
CiLogsCredentials,
@ -734,12 +734,6 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace:
default=False,
help="skip fetching data about job runs, used in --configure action (for debugging and nigthly ci)",
)
parser.add_argument(
"--rebuild-all-docker",
action="store_true",
default=False,
help="will create run config for rebuilding all dockers, used in --configure action (for nightly docker job)",
)
# FIXME: remove, not used
parser.add_argument(
"--rebuild-all-binaries",
@ -865,7 +859,7 @@ def _mark_success_action(
# there is no status for build jobs
# create dummy success to mark it as done
# FIXME: consider creating commit status for build jobs too, to treat everything the same way
CommitStatusData("success", "dummy description", "dummy_url").dump_status()
CommitStatusData(SUCCESS, "dummy description", "dummy_url").dump_status()
job_status = None
if CommitStatusData.exist():
@ -939,9 +933,7 @@ def _update_config_for_docs_only(jobs_data: dict) -> None:
}
def _configure_docker_jobs(
rebuild_all_dockers: bool, docker_digest_or_latest: bool = False
) -> Dict:
def _configure_docker_jobs(docker_digest_or_latest: bool) -> Dict:
print("::group::Docker images check")
# generate docker jobs data
docker_digester = DockerDigester()
@ -950,50 +942,33 @@ def _configure_docker_jobs(
) # 'image name - digest' mapping
images_info = docker_images_helper.get_images_info()
# a. check missing images
if not rebuild_all_dockers:
# FIXME: we need login as docker manifest inspect goes directly to one of the *.docker.com hosts instead of "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"]
# find if it's possible to use the setting of /etc/docker/daemon.json
docker_images_helper.docker_login()
missing_multi_dict = check_missing_images_on_dockerhub(imagename_digest_dict)
missing_multi = list(missing_multi_dict)
missing_amd64 = []
missing_aarch64 = []
if not docker_digest_or_latest:
# look for missing arm and amd images only among missing multiarch manifests @missing_multi_dict
# to avoid extra dockerhub api calls
missing_amd64 = list(
check_missing_images_on_dockerhub(missing_multi_dict, "amd64")
)
# FIXME: WA until full arm support: skip not supported arm images
missing_aarch64 = list(
check_missing_images_on_dockerhub(
{
im: digest
for im, digest in missing_multi_dict.items()
if not images_info[im]["only_amd64"]
},
"aarch64",
)
)
# FIXME: temporary hack, remove after transition to docker digest as tag
else:
if missing_multi:
print(
f"WARNING: Missing images {list(missing_multi)} - fallback to latest tag"
)
for image in missing_multi:
imagename_digest_dict[image] = "latest"
else:
# add all images to missing
missing_multi = list(imagename_digest_dict)
missing_amd64 = missing_multi
# FIXME: we need login as docker manifest inspect goes directly to one of the *.docker.com hosts instead of "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"]
# find if it's possible to use the setting of /etc/docker/daemon.json (https://github.com/docker/cli/issues/4484#issuecomment-1688095463)
docker_images_helper.docker_login()
missing_multi_dict = check_missing_images_on_dockerhub(imagename_digest_dict)
missing_multi = list(missing_multi_dict)
missing_amd64 = []
missing_aarch64 = []
if not docker_digest_or_latest:
# look for missing arm and amd images only among missing multiarch manifests @missing_multi_dict
# to avoid extra dockerhub api calls
missing_amd64 = list(
check_missing_images_on_dockerhub(missing_multi_dict, "amd64")
)
# FIXME: WA until full arm support: skip not supported arm images
missing_aarch64 = [
name
for name in imagename_digest_dict
if not images_info[name]["only_amd64"]
]
missing_aarch64 = list(
check_missing_images_on_dockerhub(
{
im: digest
for im, digest in missing_multi_dict.items()
if not images_info[im]["only_amd64"]
},
"aarch64",
)
)
else:
if missing_multi:
assert False, f"Missing images [{missing_multi}], cannot proceed"
print("::endgroup::")
return {
@ -1167,7 +1142,7 @@ def _update_gh_statuses_action(indata: Dict, s3: S3Helper) -> None:
if not job_status:
return
print(f"Going to re-create GH status for job [{job}] sha [{pr_info.sha}]")
assert job_status.status == "success", "BUG!"
assert job_status.status == SUCCESS, "BUG!"
commit.create_status(
state=job_status.status,
target_url=job_status.report_url,
@ -1502,9 +1477,7 @@ def main() -> int:
print(f"Got CH version for this commit: [{version}]")
docker_data = (
_configure_docker_jobs(
args.rebuild_all_docker, args.docker_digest_or_latest
)
_configure_docker_jobs(args.docker_digest_or_latest)
if not args.skip_docker
else {}
)
@ -1528,17 +1501,16 @@ def main() -> int:
else {}
)
# FIXME: Early style check manipulates with job names might be not robust with await feature
if pr_info.number != 0 and not args.docker_digest_or_latest:
# FIXME: it runs style check before docker build if possible (style-check images is not changed)
# find a way to do style check always before docker build and others
_check_and_update_for_early_style_check(jobs_data, docker_data)
if args.skip_jobs and pr_info.has_changes_in_documentation_only():
# # FIXME: Early style check manipulates with job names might be not robust with await feature
# if pr_info.number != 0:
# # FIXME: it runs style check before docker build if possible (style-check images is not changed)
# # find a way to do style check always before docker build and others
# _check_and_update_for_early_style_check(jobs_data, docker_data)
if not args.skip_jobs and pr_info.has_changes_in_documentation_only():
_update_config_for_docs_only(jobs_data)
# TODO: await pending jobs
# wait for pending jobs to be finished, await_jobs is a long blocking call if any job has to be awaited
ci_cache = CiCache(s3, jobs_data["digests"])
# awaited_jobs = ci_cache.await_jobs(jobs_data.get("jobs_to_wait", {}))
# for job in awaited_jobs:
# jobs_to_do = jobs_data["jobs_to_do"]
@ -1548,7 +1520,8 @@ def main() -> int:
# assert False, "BUG"
# set planned jobs as pending in the CI cache if on the master
if pr_info.is_master():
if pr_info.is_master() and not args.skip_jobs:
ci_cache = CiCache(s3, jobs_data["digests"])
for job in jobs_data["jobs_to_do"]:
config = CI_CONFIG.get_job_config(job)
if config.run_always or config.run_by_label:

View File

@ -296,6 +296,7 @@ stateless_check_digest = DigestConfig(
include_paths=[
"./tests/queries/0_stateless/",
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
],
exclude_files=[".md"],
@ -305,6 +306,7 @@ stateful_check_digest = DigestConfig(
include_paths=[
"./tests/queries/1_stateful/",
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
],
exclude_files=[".md"],
@ -316,6 +318,7 @@ stress_check_digest = DigestConfig(
"./tests/queries/0_stateless/",
"./tests/queries/1_stateful/",
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
],
exclude_files=[".md"],
@ -962,10 +965,6 @@ CI_CONFIG = CiConfig(
JobNames.STATELESS_TEST_ANALYZER_RELEASE: TestConfig(
Build.PACKAGE_RELEASE, job_config=JobConfig(**statless_test_common_params) # type: ignore
),
# delete?
# "Stateless tests (release, DatabaseOrdinary)": TestConfig(
# Build.PACKAGE_RELEASE, job_config=JobConfig(**statless_test_common_params) # type: ignore
# ),
JobNames.STATELESS_TEST_DB_REPL_RELEASE: TestConfig(
Build.PACKAGE_RELEASE,
job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore
@ -1112,7 +1111,7 @@ CI_CONFIG = CiConfig(
JobNames.SQL_LOGIC_TEST: TestConfig(
Build.PACKAGE_RELEASE, job_config=JobConfig(**sqllogic_test_params) # type: ignore
),
JobNames.SQL_LOGIC_TEST: TestConfig(
JobNames.SQLTEST: TestConfig(
Build.PACKAGE_RELEASE, job_config=JobConfig(**sql_test_params) # type: ignore
),
JobNames.CLCIKBENCH_TEST: TestConfig(Build.PACKAGE_RELEASE),

View File

@ -10,18 +10,14 @@ from pathlib import Path
from typing import List, Tuple
from build_download_helper import download_all_deb_packages
from clickhouse_helper import (
CiLogsCredentials,
)
from commit_status_helper import (
override_status,
)
from docker_images_helper import get_docker_image, pull_image, DockerImage
from env_helper import TEMP_PATH, REPORT_PATH
from clickhouse_helper import CiLogsCredentials
from commit_status_helper import override_status
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPORT_PATH, TEMP_PATH
from pr_info import FORCE_TESTS_LABEL, PRInfo
from report import ERROR, SUCCESS, JobReport, StatusType, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
from report import JobReport, TestResults
def get_image_name() -> str:
@ -52,7 +48,7 @@ def get_run_command(
def process_results(
result_directory: Path,
server_log_path: Path,
) -> Tuple[str, str, TestResults, List[Path]]:
) -> Tuple[StatusType, str, TestResults, List[Path]]:
test_results = [] # type: TestResults
additional_files = [] # type: List[Path]
# Just upload all files from result_directory.
@ -74,7 +70,7 @@ def process_results(
if len(status) != 1 or len(status[0]) != 2:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Invalid check_status.tsv", test_results, additional_files
return ERROR, "Invalid check_status.tsv", test_results, additional_files
state, description = status[0][0], status[0][1]
try:
@ -84,17 +80,17 @@ def process_results(
logging.info("Found %s", results_path.name)
else:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Not found test_results.tsv", test_results, additional_files
return ERROR, "Not found test_results.tsv", test_results, additional_files
except Exception as e:
return (
"error",
ERROR,
f"Cannot parse test_results.tsv ({e})",
test_results,
additional_files,
)
return state, description, test_results, additional_files
return state, description, test_results, additional_files # type: ignore
def parse_args():
@ -179,7 +175,7 @@ def main():
additional_files=[run_log_path] + additional_logs,
).dump()
if state != "success":
if state != SUCCESS:
if FORCE_TESTS_LABEL in pr_info.labels:
print(f"'{FORCE_TESTS_LABEL}' enabled, will report success")
else:

View File

@ -1,14 +1,15 @@
#!/usr/bin/env python3
from collections import defaultdict
import json
from pathlib import Path
from typing import Dict, List, Optional, Union
import csv
import json
import logging
import time
from collections import defaultdict
from dataclasses import asdict, dataclass
from pathlib import Path
from typing import Dict, List, Optional, Union
# isort: off
from github import Github
from github.Commit import Commit
from github.CommitStatus import CommitStatus
@ -17,15 +18,17 @@ from github.GithubObject import NotSet
from github.IssueComment import IssueComment
from github.Repository import Repository
from ci_config import CI_CONFIG, REQUIRED_CHECKS, CHECK_DESCRIPTIONS, CheckDescription
# isort: on
from ci_config import CHECK_DESCRIPTIONS, CI_CONFIG, REQUIRED_CHECKS, CheckDescription
from env_helper import GITHUB_JOB_URL, GITHUB_REPOSITORY, TEMP_PATH
from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL
from pr_info import SKIP_MERGEABLE_CHECK_LABEL, PRInfo
from report import (
ERROR,
FAILURE,
PENDING,
StatusType,
SUCCESS,
StatusType,
TestResult,
TestResults,
get_worst_status,
@ -64,7 +67,9 @@ class RerunHelper:
return None
def override_status(status: str, check_name: str, invert: bool = False) -> str:
def override_status(
status: StatusType, check_name: str, invert: bool = False
) -> StatusType:
test_config = CI_CONFIG.test_configs.get(check_name)
if test_config and test_config.force_tests:
return SUCCESS
@ -93,7 +98,7 @@ def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit:
def post_commit_status(
commit: Commit,
state: str,
state: StatusType,
report_url: Optional[str] = None,
description: Optional[str] = None,
check_name: Optional[str] = None,
@ -288,7 +293,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
return "".join(result)
def get_worst_state(statuses: CommitStatuses) -> str:
def get_worst_state(statuses: CommitStatuses) -> StatusType:
return get_worst_status(status.state for status in statuses)
@ -373,12 +378,12 @@ class CommitStatusData:
def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses:
"""
Squash statuses to latest state
1. context="first", state="success", update_time=1
2. context="second", state="success", update_time=2
3. context="first", stat="failure", update_time=3
1. context="first", state=SUCCESS, update_time=1
2. context="second", state=SUCCESS, update_time=2
3. context="first", stat=FAILURE, update_time=3
=========>
1. context="second", state="success"
2. context="first", stat="failure"
1. context="second", state=SUCCESS
2. context="first", stat=FAILURE
"""
filtered = {}
for status in sorted(commit.get_statuses(), key=lambda x: x.updated_at):
@ -430,7 +435,7 @@ def format_description(description: str) -> str:
def set_mergeable_check(
commit: Commit,
description: str = "",
state: StatusType = "success",
state: StatusType = SUCCESS,
) -> None:
commit.create_status(
context=MERGEABLE_NAME,

View File

@ -1,17 +1,17 @@
#!/usr/bin/env python3
from distutils.version import StrictVersion
from pathlib import Path
from typing import List, Tuple
import argparse
import logging
import subprocess
import sys
from distutils.version import StrictVersion
from pathlib import Path
from typing import List, Tuple
from build_download_helper import download_builds_filter
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import TEMP_PATH, REPORT_PATH
from report import JobReport, TestResults, TestResult
from env_helper import REPORT_PATH, TEMP_PATH
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
IMAGE_UBUNTU = "clickhouse/test-old-ubuntu"
@ -55,19 +55,19 @@ def process_result(
glibc_log_path = result_directory / "glibc.log"
test_results = process_glibc_check(glibc_log_path, max_glibc_version)
status = "success"
status = SUCCESS
description = "Compatibility check passed"
if check_glibc:
if len(test_results) > 1 or test_results[0].status != "OK":
status = "failure"
status = FAILURE
description = "glibc check failed"
if status == "success" and check_distributions:
if status == SUCCESS and check_distributions:
for operating_system in ("ubuntu:12.04", "centos:5"):
test_result = process_os_check(result_directory / operating_system)
if test_result.status != "OK":
status = "failure"
status = FAILURE
description = f"Old {operating_system} failed"
test_results += [test_result]
break
@ -178,14 +178,14 @@ def main():
)
run_commands.extend(check_distributions_commands)
state = "success"
state = SUCCESS
for run_command in run_commands:
try:
logging.info("Running command %s", run_command)
subprocess.check_call(run_command, shell=True)
except subprocess.CalledProcessError as ex:
logging.info("Exception calling command %s", ex)
state = "failure"
state = FAILURE
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
@ -215,7 +215,7 @@ def main():
additional_files=additional_logs,
).dump()
if state == "failure":
if state == FAILURE:
sys.exit(1)

View File

@ -3,24 +3,27 @@ import argparse
import json
import logging
import os
import time
import sys
import time
from pathlib import Path
from typing import List, Optional, Tuple
# isort: off
from github import Github
# isort: on
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from commit_status_helper import format_description, get_commit, post_commit_status
from env_helper import RUNNER_TEMP, GITHUB_RUN_URL
from docker_images_helper import DockerImageData, docker_login, get_images_oredered_list
from env_helper import GITHUB_RUN_URL, RUNNER_TEMP
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResults, TestResult
from report import FAILURE, SUCCESS, StatusType, TestResult, TestResults
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
from docker_images_helper import DockerImageData, docker_login, get_images_oredered_list
NAME = "Push to Dockerhub"
TEMP_PATH = Path(RUNNER_TEMP) / "docker_images_check"
@ -189,7 +192,7 @@ def main():
# additional_cache.append(str(pr_info.merged_pr))
ok_cnt = 0
status = "success"
status = SUCCESS # type: StatusType
image_tags = (
json.loads(args.image_tags)
if not os.path.isfile(args.image_tags)
@ -233,7 +236,7 @@ def main():
if all(x.status == "OK" for x in res):
ok_cnt += 1
else:
status = "failure"
status = FAILURE
break # No need to continue with next images
description = format_description(
@ -268,7 +271,7 @@ def main():
ch_helper = ClickHouseHelper()
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if status == "failure":
if status == FAILURE:
sys.exit(1)

View File

@ -5,24 +5,23 @@ import json
import logging
import os
import subprocess
import sys
from typing import List, Tuple
# isort: off
from github import Github
from clickhouse_helper import (
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
)
# isort: on
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from commit_status_helper import format_description, get_commit, post_commit_status
from docker_images_helper import docker_login, get_images_oredered_list
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResult
from report import FAILURE, SUCCESS, StatusType, TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from upload_result_helper import upload_results
from docker_images_helper import docker_login, get_images_oredered_list
NAME = "Push multi-arch images to Dockerhub"
@ -149,29 +148,35 @@ def main():
else json.load(open(args.missing_images))
)
test_results = []
status = "success"
status = SUCCESS # type: StatusType
ok_cnt, fail_cnt = 0, 0
images = get_images_oredered_list()
for image_obj in images:
if image_obj.repo not in missing_images:
continue
tag = image_tags[image_obj.repo]
if image_obj.only_amd64:
# FIXME: WA until full arm support
tags = [f"{tag}-{arch}" for arch in archs if arch != "aarch64"]
else:
tags = [f"{tag}-{arch}" for arch in archs]
manifest, test_result = create_manifest(image_obj.repo, tag, tags, args.push)
test_results.append(TestResult(manifest, test_result))
# 1. update multiarch latest manifest for every image
if args.set_latest:
manifest, test_result = create_manifest(
image_obj.repo, "latest", tags, args.push
)
test_results.append(TestResult(manifest, test_result))
# 2. skip manifest create if not missing
if image_obj.repo not in missing_images:
continue
# 3. created image:digest multiarch manifest for changed images only
manifest, test_result = create_manifest(image_obj.repo, tag, tags, args.push)
test_results.append(TestResult(manifest, test_result))
if test_result != "OK":
status = "failure"
status = FAILURE
fail_cnt += 1
else:
ok_cnt += 1
@ -207,7 +212,7 @@ def main():
)
ch_helper = ClickHouseHelper()
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if status == "failure":
if status == FAILURE:
sys.exit(1)

View File

@ -6,25 +6,26 @@ import json
import logging
import sys
import time
from os import makedirs
from os import path as p
from pathlib import Path
from os import path as p, makedirs
from typing import Dict, List
from build_check import get_release_or_pr
from build_download_helper import read_build_urls
from docker_images_helper import DockerImageData, docker_login
from env_helper import (
GITHUB_RUN_URL,
REPORT_PATH,
TEMP_PATH,
S3_BUILDS_BUCKET,
S3_DOWNLOAD,
TEMP_PATH,
)
from git_helper import Git
from pr_info import PRInfo
from report import JobReport, TestResults, TestResult
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
from build_download_helper import read_build_urls
from version_helper import (
ClickHouseVersion,
get_tagged_versions,
@ -378,7 +379,7 @@ def main():
docker_login()
logging.info("Following tags will be created: %s", ", ".join(tags))
status = "success"
status = SUCCESS
test_results = [] # type: TestResults
for os in args.os:
for tag in tags:
@ -388,7 +389,7 @@ def main():
)
)
if test_results[-1].status != "OK":
status = "failure"
status = FAILURE
pr_info = pr_info or PRInfo()
description = f"Processed tags: {', '.join(tags)}"
@ -401,7 +402,7 @@ def main():
additional_files=[],
).dump()
if status != "success":
if status != SUCCESS:
sys.exit(1)

View File

@ -8,7 +8,7 @@ from pathlib import Path
from docker_images_helper import get_docker_image, pull_image
from env_helper import REPO_COPY, TEMP_PATH
from pr_info import PRInfo
from report import JobReport, TestResult, TestResults
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -49,7 +49,7 @@ def main():
JobReport(
description="No changes in docs",
test_results=[],
status="success",
status=SUCCESS,
start_time=stopwatch.start_time_str,
duration=stopwatch.duration_seconds,
additional_files=[],
@ -79,11 +79,11 @@ def main():
retcode = process.wait()
if retcode == 0:
logging.info("Run successfully")
status = "success"
status = SUCCESS
description = "Docs check passed"
else:
description = "Docs check failed (non zero exit code)"
status = "failure"
status = FAILURE
logging.info("Run failed")
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
@ -92,7 +92,7 @@ def main():
if not any(test_output.iterdir()):
logging.error("No output files after docs check")
description = "No output files after docs check"
status = "failure"
status = FAILURE
else:
for p in test_output.iterdir():
additional_files.append(p)
@ -101,9 +101,9 @@ def main():
if "ERROR" in line:
test_results.append(TestResult(line.split(":")[-1], "FAIL"))
if test_results:
status = "failure"
status = FAILURE
description = "Found errors in docs"
elif status != "failure":
elif status != FAILURE:
test_results.append(TestResult("No errors found", "OK"))
else:
test_results.append(TestResult("Non zero exit code", "FAIL"))
@ -117,7 +117,7 @@ def main():
additional_files=additional_files,
).dump()
if status == "failure":
if status == FAILURE:
sys.exit(1)

View File

@ -11,7 +11,15 @@ from typing import Tuple
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPO_COPY, S3_BUILDS_BUCKET, TEMP_PATH
from pr_info import FORCE_TESTS_LABEL, PRInfo
from report import JobReport, TestResult, TestResults, read_test_results
from report import (
ERROR,
FAILURE,
SUCCESS,
JobReport,
TestResult,
TestResults,
read_test_results,
)
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -56,16 +64,16 @@ def process_results(result_directory: Path) -> Tuple[str, str, TestResults]:
status = list(csv.reader(status_file, delimiter="\t"))
if len(status) != 1 or len(status[0]) != 2:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Invalid check_status.tsv", test_results
return ERROR, "Invalid check_status.tsv", test_results
state, description = status[0][0], status[0][1]
try:
results_path = result_directory / "test_results.tsv"
test_results = read_test_results(results_path)
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results
return ERROR, "Empty test_results.tsv", test_results
except Exception as e:
return ("error", f"Cannot parse test_results.tsv ({e})", test_results)
return (ERROR, f"Cannot parse test_results.tsv ({e})", test_results)
return state, description, test_results
@ -149,25 +157,25 @@ def main():
test_results = [] # type: TestResults
if "submodule_log.txt" not in test_output_files:
description = "Cannot clone repository"
state = "failure"
state = FAILURE
elif "cmake_log.txt" not in test_output_files:
description = "Cannot fetch submodules"
state = "failure"
state = FAILURE
elif "build_log.txt" not in test_output_files:
description = "Cannot finish cmake"
state = "failure"
state = FAILURE
elif "install_log.txt" not in test_output_files:
description = "Cannot build ClickHouse"
state = "failure"
state = FAILURE
elif not test_log_exists and not test_result_exists:
description = "Cannot install or start ClickHouse"
state = "failure"
state = FAILURE
else:
state, description, test_results = process_results(output_path)
if timeout_expired:
test_results.append(TestResult.create_check_timeout_expired(args.timeout))
state = "failure"
state = FAILURE
description = test_results[-1].name
JobReport(
@ -181,8 +189,8 @@ def main():
).dump()
# Refuse other checks to run if fast test failed
if state != "success":
if state == "error":
if state != SUCCESS:
if state == ERROR:
print("The status is 'error', report failure disregard the labels")
sys.exit(1)
elif FORCE_TESTS_LABEL in pr_info.labels:

View File

@ -1,7 +1,11 @@
#!/usr/bin/env python3
import logging
# isort: off
from github import Github
# isort: on
from commit_status_helper import (
CI_STATUS_NAME,
get_commit,
@ -11,6 +15,7 @@ from commit_status_helper import (
)
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import PENDING, SUCCESS
def main():
@ -31,10 +36,10 @@ def main():
return
# Take the latest status
status = statuses[-1]
if status.state == "pending":
if status.state == PENDING:
post_commit_status(
commit,
"success",
SUCCESS,
status.target_url,
"All checks finished",
CI_STATUS_NAME,

View File

@ -1,18 +1,21 @@
#!/usr/bin/env python3
import argparse
import atexit
import csv
import logging
import os
import re
import subprocess
import sys
import atexit
from pathlib import Path
from typing import List, Tuple
# isort: off
from github import Github
# isort: on
from build_download_helper import download_all_deb_packages
from clickhouse_helper import (
CiLogsCredentials,
@ -26,12 +29,12 @@ from commit_status_helper import (
post_commit_status_to_file,
update_mergeable_check,
)
from docker_images_helper import DockerImage, pull_image, get_docker_image
from docker_images_helper import DockerImage, get_docker_image, pull_image
from download_release_packages import download_last_release
from env_helper import REPORT_PATH, TEMP_PATH, REPO_COPY
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from get_robot_token import get_best_robot_token
from pr_info import FORCE_TESTS_LABEL, PRInfo
from report import TestResults, read_test_results
from report import ERROR, SUCCESS, StatusType, TestResults, read_test_results
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -152,7 +155,7 @@ def get_tests_to_run(pr_info: PRInfo) -> List[str]:
def process_results(
result_directory: Path,
server_log_path: Path,
) -> Tuple[str, str, TestResults, List[Path]]:
) -> Tuple[StatusType, str, TestResults, List[Path]]:
test_results = [] # type: TestResults
additional_files = []
# Just upload all files from result_directory.
@ -174,7 +177,7 @@ def process_results(
if len(status) != 1 or len(status[0]) != 2:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Invalid check_status.tsv", test_results, additional_files
return ERROR, "Invalid check_status.tsv", test_results, additional_files
state, description = status[0][0], status[0][1]
try:
@ -184,20 +187,20 @@ def process_results(
logging.info("Found test_results.tsv")
else:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Not found test_results.tsv", test_results, additional_files
return ERROR, "Not found test_results.tsv", test_results, additional_files
test_results = read_test_results(results_path)
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files
return ERROR, "Empty test_results.tsv", test_results, additional_files
except Exception as e:
return (
"error",
ERROR,
f"Cannot parse test_results.tsv ({e})",
test_results,
additional_files,
)
return state, description, test_results, additional_files
return state, description, test_results, additional_files # type: ignore
def parse_args():
@ -262,7 +265,7 @@ def main():
post_commit_status_to_file(
post_commit_path,
f"Skipped (no pr-bugfix in {pr_info.labels})",
"success",
SUCCESS,
"null",
)
logging.info("Skipping '%s' (no pr-bugfix in %s)", check_name, pr_info.labels)
@ -283,7 +286,7 @@ def main():
if run_changed_tests:
tests_to_run = get_tests_to_run(pr_info)
if not tests_to_run:
state = override_status("success", check_name, validate_bugfix_check)
state = override_status(SUCCESS, check_name, validate_bugfix_check)
if args.post_commit_status == "commit_status":
post_commit_status(
commit,
@ -415,7 +418,7 @@ def main():
)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if state != "success":
if state != SUCCESS:
if FORCE_TESTS_LABEL in pr_info.labels:
print(f"'{FORCE_TESTS_LABEL}' enabled, will report success")
else:

View File

@ -25,7 +25,14 @@ from get_robot_token import get_best_robot_token
from github_helper import GitHub
from integration_test_images import IMAGES
from pr_info import PRInfo
from report import ERROR, TestResult, TestResults, read_test_results
from report import (
ERROR,
SUCCESS,
StatusType,
TestResult,
TestResults,
read_test_results,
)
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -84,7 +91,7 @@ def get_env_for_runner(
def process_results(
result_directory: Path,
) -> Tuple[str, str, TestResults, List[Path]]:
) -> Tuple[StatusType, str, TestResults, List[Path]]:
test_results = [] # type: TestResults
additional_files = []
# Just upload all files from result_directory.
@ -102,23 +109,23 @@ def process_results(
if len(status) != 1 or len(status[0]) != 2:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Invalid check_status.tsv", test_results, additional_files
return ERROR, "Invalid check_status.tsv", test_results, additional_files
state, description = status[0][0], status[0][1]
try:
results_path = result_directory / "test_results.tsv"
test_results = read_test_results(results_path, False)
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files
return ERROR, "Empty test_results.tsv", test_results, additional_files
except Exception as e:
return (
"error",
ERROR,
f"Cannot parse test_results.tsv ({e})",
test_results,
additional_files,
)
return state, description, test_results, additional_files
return state, description, test_results, additional_files # type: ignore
def parse_args():
@ -182,7 +189,7 @@ def main():
post_commit_status_to_file(
post_commit_path,
f"Skipped (no pr-bugfix in {pr_info.labels})",
"success",
SUCCESS,
"null",
)
logging.info("Skipping '%s' (no pr-bugfix in '%s')", check_name, pr_info.labels)
@ -315,7 +322,7 @@ def main():
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if state == "failure":
if state != SUCCESS:
sys.exit(1)

View File

@ -5,29 +5,26 @@ import logging
import os
import sys
import time
from pathlib import Path
from typing import Any, List
import boto3 # type: ignore
import requests # type: ignore
from build_download_helper import (
download_build_with_progress,
get_build_name_for_check,
read_build_urls,
)
from compress_files import compress_fast
from env_helper import REPO_COPY, REPORT_PATH, S3_URL, TEMP_PATH, S3_BUILDS_BUCKET
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, S3_URL, TEMP_PATH
from get_robot_token import get_parameter_from_ssm
from git_helper import git_runner
from pr_info import PRInfo
from report import JobReport, TestResults, TestResult
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from ssh import SSHKey
from stopwatch import Stopwatch
from tee_popen import TeePopen
JEPSEN_GROUP_NAME = "jepsen_group"
KEEPER_DESIRED_INSTANCE_COUNT = 3
@ -263,21 +260,21 @@ def main():
else:
logging.info("Run failed")
status = "success"
status = SUCCESS
description = "No invalid analysis found ヽ(‘ー`)"
jepsen_log_path = result_path / "jepsen_run_all_tests.log"
additional_data = []
try:
test_result = _parse_jepsen_output(jepsen_log_path)
if any(r.status == "FAIL" for r in test_result):
status = "failure"
status = FAILURE
description = "Found invalid analysis (ノಥ益ಥ)ノ ┻━┻"
compress_fast(result_path / "store", result_path / "jepsen_store.tar.zst")
additional_data.append(result_path / "jepsen_store.tar.zst")
except Exception as ex:
print("Exception", ex)
status = "failure"
status = FAILURE
description = "No Jepsen output log"
test_result = [TestResult("No Jepsen output log", "FAIL")]

View File

@ -43,6 +43,7 @@ TRUSTED_CONTRIBUTORS = {
"tsolodov", # ClickHouse, Inc
"kitaisreal",
"k-morozov", # Konstantin Morozov, Yandex Cloud
"justindeguzman", # ClickHouse, Inc
]
}

View File

@ -7,10 +7,11 @@ import os
from commit_status_helper import get_commit, post_commit_status
from env_helper import GITHUB_JOB_URL
from get_robot_token import get_best_robot_token
from git_helper import commit as commit_arg
from github_helper import GitHub
from pr_info import PRInfo
from release import RELEASE_READY_STATUS
from git_helper import commit as commit_arg
from report import SUCCESS
def main():
@ -50,7 +51,7 @@ def main():
gh.get_rate_limit()
post_commit_status(
commit,
"success",
SUCCESS,
url,
description,
RELEASE_READY_STATUS,

View File

@ -4,21 +4,23 @@
import argparse
import logging
from datetime import datetime
from os import getenv
from pprint import pformat
from typing import Dict, List
# isort: off
from github.PaginatedList import PaginatedList
from github.PullRequestReview import PullRequestReview
from github.WorkflowRun import WorkflowRun
# isort: on
from commit_status_helper import get_commit_filtered_statuses
from get_robot_token import get_best_robot_token
from github_helper import GitHub, NamedUser, PullRequest, Repository
from pr_info import PRInfo
from report import SUCCESS
# The team name for accepted approvals
TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core")
@ -269,7 +271,7 @@ def main():
failed_statuses = [
status.context
for status in get_commit_filtered_statuses(commit)
if status.state != "success"
if status.state != SUCCESS
]
if failed_statuses:
logging.warning(

View File

@ -1,35 +1,38 @@
#!/usr/bin/env python3
import os
import logging
import sys
import json
import subprocess
import traceback
import logging
import os
import re
import subprocess
import sys
import traceback
from pathlib import Path
# isort: off
from github import Github
from commit_status_helper import get_commit
# isort: on
from build_download_helper import download_builds_filter
from ci_config import CI_CONFIG
from docker_images_helper import pull_image, get_docker_image
from clickhouse_helper import get_instance_id, get_instance_type
from commit_status_helper import get_commit
from docker_images_helper import get_docker_image, pull_image
from env_helper import (
GITHUB_EVENT_PATH,
GITHUB_RUN_URL,
REPO_COPY,
REPORT_PATH,
S3_BUILDS_BUCKET,
S3_DOWNLOAD,
TEMP_PATH,
REPORT_PATH,
)
from get_robot_token import get_best_robot_token, get_parameter_from_ssm
from pr_info import PRInfo
from tee_popen import TeePopen
from clickhouse_helper import get_instance_type, get_instance_id
from report import FAILURE, SUCCESS, JobReport
from stopwatch import Stopwatch
from build_download_helper import download_builds_filter
from report import SUCCESS, JobReport
from tee_popen import TeePopen
IMAGE_NAME = "clickhouse/performance-comparison"
@ -225,18 +228,18 @@ def main():
# TODO: Remove me, always green mode for the first time, unless errors
status = SUCCESS
if "errors" in message.lower() or too_many_slow(message.lower()):
status = "failure"
status = FAILURE
# TODO: Remove until here
except Exception:
traceback.print_exc()
status = "failure"
status = FAILURE
message = "Failed to parse the report."
if not status:
status = "failure"
status = FAILURE
message = "No status in report."
elif not message:
status = "failure"
status = FAILURE
message = "No message in report."
JobReport(

View File

@ -18,6 +18,7 @@ from contextlib import contextmanager
from typing import Any, Final, Iterator, List, Optional, Tuple
from git_helper import Git, commit, release_branch
from report import SUCCESS
from version_helper import (
FILE_WITH_VERSION_PATH,
GENERATED_CONTRIBUTORS,
@ -142,7 +143,7 @@ class Release:
for status in statuses:
if status["context"] == RELEASE_READY_STATUS:
if not status["state"] == "success":
if not status["state"] == SUCCESS:
raise Exception(
f"the status {RELEASE_READY_STATUS} is {status['state']}"
", not success"

View File

@ -1,6 +1,12 @@
# -*- coding: utf-8 -*-
import csv
import datetime
import json
import logging
import os
from ast import literal_eval
from dataclasses import asdict, dataclass
from html import escape
from pathlib import Path
from typing import (
Dict,
@ -13,18 +19,11 @@ from typing import (
Tuple,
Union,
)
from html import escape
import csv
import datetime
import json
import logging
import os
from build_download_helper import get_gh_api
from ci_config import BuildConfig, CI_CONFIG
from ci_config import CI_CONFIG, BuildConfig
from env_helper import REPORT_PATH, TEMP_PATH
logger = logging.getLogger(__name__)
ERROR: Final = "error"
@ -36,26 +35,28 @@ OK: Final = "OK"
FAIL: Final = "FAIL"
StatusType = Literal["error", "failure", "pending", "success"]
STATUSES = [ERROR, FAILURE, PENDING, SUCCESS] # type: List[StatusType]
# The order of statuses from the worst to the best
_STATES = {ERROR: 0, FAILURE: 1, PENDING: 2, SUCCESS: 3}
def _state_rank(status: str) -> int:
"return the index of status or index of SUCCESS in case of wrong status"
try:
return STATUSES.index(status) # type: ignore
except ValueError:
return 3
def get_worst_status(statuses: Iterable[str]) -> str:
worst_status = None
def get_worst_status(statuses: Iterable[str]) -> StatusType:
worst_status = SUCCESS # type: StatusType
for status in statuses:
if _STATES.get(status) is None:
continue
if worst_status is None:
worst_status = status
continue
if _STATES.get(status) < _STATES.get(worst_status):
worst_status = status
ind = _state_rank(status)
if ind < _state_rank(worst_status):
worst_status = STATUSES[ind]
if worst_status == ERROR:
break
if worst_status is None:
return ""
return worst_status

View File

@ -27,7 +27,7 @@ from lambda_shared_package.lambda_shared.pr import (
check_pr_description,
)
from pr_info import FORCE_TESTS_LABEL, PRInfo
from report import FAILURE
from report import FAILURE, PENDING
TRUSTED_ORG_IDS = {
54801242, # clickhouse
@ -188,7 +188,7 @@ def main():
print("::notice ::Can run")
post_commit_status(
commit,
"pending",
PENDING,
ci_report_url,
description,
CI_STATUS_NAME,

View File

@ -7,12 +7,9 @@ import sys
from pathlib import Path
from build_download_helper import get_build_name_for_check, read_build_urls
from docker_images_helper import DockerImage, pull_image, get_docker_image
from env_helper import (
REPORT_PATH,
TEMP_PATH,
)
from report import JobReport, TestResults, TestResult
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPORT_PATH, TEMP_PATH
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -94,7 +91,7 @@ def main():
paths += [workspace_path / f"{t}.err" for t in tests]
paths += [workspace_path / f"{t}.out" for t in tests]
status = "success"
status = SUCCESS
test_results = [] # type: TestResults
# Try to get status message saved by the SQLancer
try:
@ -109,7 +106,7 @@ def main():
with open(workspace_path / "description.txt", "r", encoding="utf-8") as desc_f:
description = desc_f.readline().rstrip("\n")
except:
status = "failure"
status = FAILURE
description = "Task failed: $?=" + str(retcode)
if not test_results:

View File

@ -10,22 +10,22 @@ from typing import Tuple
from build_download_helper import download_all_deb_packages
from commit_status_helper import override_status
from docker_images_helper import DockerImage, pull_image, get_docker_image
from env_helper import REPORT_PATH, TEMP_PATH, REPO_COPY
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from report import (
OK,
FAIL,
ERROR,
FAIL,
OK,
SUCCESS,
JobReport,
TestResults,
StatusType,
TestResult,
TestResults,
read_test_results,
)
from stopwatch import Stopwatch
from tee_popen import TeePopen
NO_CHANGES_MSG = "Nothing to run"
IMAGE_NAME = "clickhouse/sqllogic-test"
@ -47,7 +47,7 @@ def get_run_command(
)
def read_check_status(result_folder: Path) -> Tuple[str, str]:
def read_check_status(result_folder: Path) -> Tuple[StatusType, str]:
status_path = result_folder / "check_status.tsv"
if not status_path.exists():
return ERROR, "Not found check_status.tsv"
@ -60,9 +60,9 @@ def read_check_status(result_folder: Path) -> Tuple[str, str]:
if len(row) != 2:
return ERROR, "Invalid check_status.tsv"
if row[0] != SUCCESS:
return row[0], row[1]
return row[0], row[1] # type: ignore
return status_rows[-1][0], status_rows[-1][1]
return status_rows[-1][0], status_rows[-1][1] # type: ignore
def parse_args() -> argparse.Namespace:
@ -172,7 +172,7 @@ def main():
)
)
# Until it pass all tests, do not block CI, report "success"
# Until it pass all tests, do not block CI, report SUCCESS
assert description is not None
# FIXME: force SUCCESS until all cases are fixed
status = SUCCESS

View File

@ -1,21 +1,16 @@
#!/usr/bin/env python3
import logging
import subprocess
import os
import subprocess
import sys
from pathlib import Path
from typing import Dict
from build_download_helper import get_build_name_for_check, read_build_urls
from docker_images_helper import pull_image, get_docker_image
from env_helper import (
REPORT_PATH,
TEMP_PATH,
)
from docker_images_helper import get_docker_image, pull_image
from env_helper import REPORT_PATH, TEMP_PATH
from pr_info import PRInfo
from report import JobReport, TestResult
from report import SUCCESS, JobReport, TestResult
from stopwatch import Stopwatch
IMAGE_NAME = "clickhouse/sqltest"
@ -98,7 +93,7 @@ def main():
"report.html": workspace_path / "report.html",
"test.log": workspace_path / "test.log",
}
status = "success"
status = SUCCESS
description = "See the report"
test_results = [TestResult(description, "OK")]

View File

@ -10,11 +10,10 @@ from typing import List, Tuple
from build_download_helper import download_all_deb_packages
from clickhouse_helper import CiLogsCredentials
from docker_images_helper import DockerImage, pull_image, get_docker_image
from env_helper import REPORT_PATH, TEMP_PATH, REPO_COPY
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from pr_info import PRInfo
from report import JobReport, TestResult, TestResults, read_test_results
from report import ERROR, JobReport, TestResult, TestResults, read_test_results
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -89,7 +88,7 @@ def process_results(
status = list(csv.reader(status_file, delimiter="\t"))
if len(status) != 1 or len(status[0]) != 2:
return "error", "Invalid check_status.tsv", test_results, additional_files
return ERROR, "Invalid check_status.tsv", test_results, additional_files
state, description = status[0][0], status[0][1]
try:
@ -99,7 +98,7 @@ def process_results(
raise Exception("Empty results")
except Exception as e:
return (
"error",
ERROR,
f"Cannot parse test_results.tsv ({e})",
test_results,
additional_files,

View File

@ -12,7 +12,7 @@ from docker_images_helper import get_docker_image, pull_image
from env_helper import REPO_COPY, TEMP_PATH
from git_helper import GIT_PREFIX, git_runner
from pr_info import PRInfo
from report import JobReport, TestResults, read_test_results
from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results
from ssh import SSHKey
from stopwatch import Stopwatch
@ -36,7 +36,7 @@ def process_result(
status = list(csv.reader(status_file, delimiter="\t"))
if len(status) != 1 or len(status[0]) != 2:
logging.info("Files in result folder %s", os.listdir(result_directory))
return "error", "Invalid check_status.tsv", test_results, additional_files
return ERROR, "Invalid check_status.tsv", test_results, additional_files
state, description = status[0][0], status[0][1]
try:
@ -47,8 +47,8 @@ def process_result(
return state, description, test_results, additional_files
except Exception:
if state == "success":
state, description = "error", "Failed to read test_results.tsv"
if state == SUCCESS:
state, description = ERROR, "Failed to read test_results.tsv"
return state, description, test_results, additional_files
@ -161,7 +161,7 @@ def main():
additional_files=additional_files,
).dump()
if state in ["error", "failure"]:
if state in [ERROR, FAILURE]:
print(f"Style check failed: [{description}]")
sys.exit(1)

View File

@ -3,15 +3,15 @@
import json
import logging
import os
import sys
import subprocess
import sys
from pathlib import Path
from typing import Tuple
from build_download_helper import download_unit_tests
from docker_images_helper import pull_image, get_docker_image
from docker_images_helper import get_docker_image, pull_image
from env_helper import REPORT_PATH, TEMP_PATH
from report import ERROR, FAILURE, FAIL, OK, SUCCESS, JobReport, TestResults, TestResult
from report import ERROR, FAIL, FAILURE, OK, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -104,7 +104,7 @@ def process_results(
if "failures" in test_case:
raw_logs = ""
for failure in test_case["failures"]:
raw_logs += failure["failure"]
raw_logs += failure[FAILURE]
if (
"Segmentation fault" in raw_logs # type: ignore
and SEGFAULT not in description
@ -205,7 +205,7 @@ def main():
additional_files=additional_files,
).dump()
if state == "failure":
if state == FAILURE:
sys.exit(1)

View File

@ -7,9 +7,9 @@ INSERT INTO aggregating_00191 (k, u) SELECT intDiv(number, 100) AS k, uniqState(
SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL order by k;
OPTIMIZE TABLE aggregating_00191;
OPTIMIZE TABLE aggregating_00191 FINAL;
SELECT k, finalizeAggregation(u) FROM aggregating_00191;
SELECT k, finalizeAggregation(u) FROM aggregating_00191 order by k;
SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL order by k;
DROP TABLE aggregating_00191;

View File

@ -1,3 +1,5 @@
SET default_table_engine = 'None';
CREATE TABLE table_02184 (x UInt8); --{serverError 119}
SET default_table_engine = 'Log';
CREATE TABLE table_02184 (x UInt8);

View File

@ -47,24 +47,24 @@ SELECT sum(uint64) + (1 * count(uint64))
FROM test_table
WHERE ((uint64 + 1) AS i) > 0
EXPLAIN SYNTAX (SELECT sum(uint64 + 1) AS j from test_table having j > 0);
SELECT sum(uint64) + (1 * count(uint64))
SELECT sum(uint64) + (1 * count(uint64)) AS j
FROM test_table
HAVING (sum(uint64) + (1 * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(uint64 + 1 AS i) j from test_table where i > 0 having j > 0);
SELECT sum(uint64) + (1 * count(uint64))
SELECT sum(uint64) + (1 * count(uint64)) AS j
FROM test_table
WHERE ((uint64 + 1) AS i) > 0
HAVING (sum(uint64) + (1 * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum((uint64 AS m) + (1 AS n)) j from test_table where m > 0 and n > 0 having j > 0);
SELECT sum(uint64) + ((1 AS n) * count(uint64))
SELECT sum(uint64) + ((1 AS n) * count(uint64)) AS j
FROM test_table
WHERE ((uint64 AS m) > 0) AND (n > 0)
HAVING (sum(uint64) + (n * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(((uint64 AS m) + (1 AS n)) AS i) j from test_table where m > 0 and n > 0 and i > 0 having j > 0);
SELECT sum(uint64) + ((1 AS n) * count(uint64))
SELECT sum(uint64) + ((1 AS n) * count(uint64)) AS j
FROM test_table
WHERE ((uint64 AS m) > 0) AND (n > 0) AND (((m + n) AS i) > 0)
HAVING (sum(uint64) + (n * count(uint64))) > 0
HAVING j > 0
SELECT sum(1 + uint64 AS i) from test_table where i > 0;
20
SELECT sum(1 + uint64) AS j from test_table having j > 0;
@ -80,24 +80,24 @@ SELECT (1 * count(uint64)) + sum(uint64)
FROM test_table
WHERE ((1 + uint64) AS i) > 0
EXPLAIN SYNTAX (SELECT sum(1 + uint64) AS j from test_table having j > 0);
SELECT (1 * count(uint64)) + sum(uint64)
SELECT (1 * count(uint64)) + sum(uint64) AS j
FROM test_table
HAVING ((1 * count(uint64)) + sum(uint64)) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(1 + uint64 AS i) j from test_table where i > 0 having j > 0);
SELECT (1 * count(uint64)) + sum(uint64)
SELECT (1 * count(uint64)) + sum(uint64) AS j
FROM test_table
WHERE ((1 + uint64) AS i) > 0
HAVING ((1 * count(uint64)) + sum(uint64)) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum((1 AS m) + (uint64 AS n)) j from test_table where m > 0 and n > 0 having j > 0);
SELECT ((1 AS m) * count(uint64)) + sum(uint64)
SELECT ((1 AS m) * count(uint64)) + sum(uint64) AS j
FROM test_table
WHERE (m > 0) AND ((uint64 AS n) > 0)
HAVING ((m * count(uint64)) + sum(uint64)) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(((1 AS m) + (uint64 AS n)) AS i) j from test_table where m > 0 and n > 0 and i > 0 having j > 0);
SELECT ((1 AS m) * count(uint64)) + sum(uint64)
SELECT ((1 AS m) * count(uint64)) + sum(uint64) AS j
FROM test_table
WHERE (m > 0) AND ((uint64 AS n) > 0) AND (((m + n) AS i) > 0)
HAVING ((m * count(uint64)) + sum(uint64)) > 0
HAVING j > 0
SELECT sum(uint64 - 1 AS i) from test_table where i > 0;
10
SELECT sum(uint64 - 1) AS j from test_table having j > 0;
@ -113,24 +113,24 @@ SELECT sum(uint64) - (1 * count(uint64))
FROM test_table
WHERE ((uint64 - 1) AS i) > 0
EXPLAIN SYNTAX (SELECT sum(uint64 - 1) AS j from test_table having j > 0);
SELECT sum(uint64) - (1 * count(uint64))
SELECT sum(uint64) - (1 * count(uint64)) AS j
FROM test_table
HAVING (sum(uint64) - (1 * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(uint64 - 1 AS i) j from test_table where i > 0 having j > 0);
SELECT sum(uint64) - (1 * count(uint64))
SELECT sum(uint64) - (1 * count(uint64)) AS j
FROM test_table
WHERE ((uint64 - 1) AS i) > 0
HAVING (sum(uint64) - (1 * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum((uint64 AS m) - (1 AS n)) j from test_table where m > 0 and n > 0 having j > 0);
SELECT sum(uint64) - ((1 AS n) * count(uint64))
SELECT sum(uint64) - ((1 AS n) * count(uint64)) AS j
FROM test_table
WHERE ((uint64 AS m) > 0) AND (n > 0)
HAVING (sum(uint64) - (n * count(uint64))) > 0
HAVING j > 0
EXPLAIN SYNTAX (SELECT sum(((uint64 AS m) - (1 AS n)) AS i) j from test_table where m > 0 and n > 0 and i > 0 having j > 0);
SELECT sum(uint64) - ((1 AS n) * count(uint64))
SELECT sum(uint64) - ((1 AS n) * count(uint64)) AS j
FROM test_table
WHERE ((uint64 AS m) > 0) AND (n > 0) AND (((m - n) AS i) > 0)
HAVING (sum(uint64) - (n * count(uint64))) > 0
HAVING j > 0
SELECT sum(1 - uint64 AS i) from test_table;
-10
SELECT sum(1 - uint64) AS j from test_table;
@ -146,24 +146,24 @@ SELECT (1 * count(uint64)) - sum(uint64)
FROM test_table
WHERE ((1 - uint64) AS i) > 0
EXPLAIN SYNTAX (SELECT sum(1 - uint64) AS j from test_table having j < 0);
SELECT (1 * count(uint64)) - sum(uint64)
SELECT (1 * count(uint64)) - sum(uint64) AS j
FROM test_table
HAVING ((1 * count(uint64)) - sum(uint64)) < 0
HAVING j < 0
EXPLAIN SYNTAX (SELECT sum(1 - uint64 AS i) j from test_table where i > 0 having j < 0);
SELECT (1 * count(uint64)) - sum(uint64)
SELECT (1 * count(uint64)) - sum(uint64) AS j
FROM test_table
WHERE ((1 - uint64) AS i) > 0
HAVING ((1 * count(uint64)) - sum(uint64)) < 0
HAVING j < 0
EXPLAIN SYNTAX (SELECT sum((1 AS m) - (uint64 AS n)) j from test_table where m > 0 and n > 0 having j < 0);
SELECT ((1 AS m) * count(uint64)) - sum(uint64)
SELECT ((1 AS m) * count(uint64)) - sum(uint64) AS j
FROM test_table
WHERE (m > 0) AND ((uint64 AS n) > 0)
HAVING ((m * count(uint64)) - sum(uint64)) < 0
HAVING j < 0
EXPLAIN SYNTAX (SELECT sum(((1 AS m) - (uint64 AS n)) AS i) j from test_table where m > 0 and n > 0 and i < 0 having j < 0);
SELECT ((1 AS m) * count(uint64)) - sum(uint64)
SELECT ((1 AS m) * count(uint64)) - sum(uint64) AS j
FROM test_table
WHERE (m > 0) AND ((uint64 AS n) > 0) AND (((m - n) AS i) < 0)
HAVING ((m * count(uint64)) - sum(uint64)) < 0
HAVING j < 0
SELECT sum(uint64 + 2.11) From test_table;
25.549999999999997
SELECT sum(2.11 + uint64) From test_table;
@ -474,3 +474,11 @@ FROM test_table
EXPLAIN SYNTAX (SELECT (2 * count(decimal32) - sum(decimal32)) + (3 * count(decimal32) - sum(decimal32)) From test_table);
SELECT ((2 * count(decimal32)) - sum(decimal32)) + ((3 * count(decimal32)) - sum(decimal32))
FROM test_table
-- https://github.com/ClickHouse/ClickHouse/issues/59414
SELECT sum(uint64 + 2) as j, j + 5 as t from test_table;
25 30
EXPLAIN SYNTAX SELECT sum(uint64 + 2) as j, j + 5 as t from test_table;
SELECT
sum(uint64) + (2 * count(uint64)) AS j,
j + 5 AS t
FROM test_table

View File

@ -204,6 +204,11 @@ EXPLAIN SYNTAX (SELECT (sum(decimal32) + 2 * count(decimal32)) - (sum(decimal32)
EXPLAIN SYNTAX (SELECT (sum(decimal32) - 2 * count(decimal32)) + (sum(decimal32) - 3 * count(decimal32)) From test_table);
EXPLAIN SYNTAX (SELECT (sum(decimal32) - 2 * count(decimal32)) - (sum(decimal32) - 3 * count(decimal32)) From test_table);
EXPLAIN SYNTAX (SELECT (2 * count(decimal32) - sum(decimal32)) + (3 * count(decimal32) - sum(decimal32)) From test_table);
-- https://github.com/ClickHouse/ClickHouse/issues/59414
SELECT sum(uint64 + 2) as j, j + 5 as t from test_table;
EXPLAIN SYNTAX SELECT sum(uint64 + 2) as j, j + 5 as t from test_table;
-- { echoOff }
DROP TABLE IF EXISTS test_table;

View File

@ -0,0 +1,6 @@
1
1
1
1
1
1

View File

@ -0,0 +1,17 @@
-- Tags: zookeeper, no-parallel, no-fasttest, long
SET allow_unrestricted_reads_from_keeper = 'false';
SELECT count() > 0 FROM system.zookeeper; -- { serverError BAD_ARGUMENTS }
SELECT count() > 0 FROM system.zookeeper WHERE name LIKE '%_%'; -- { serverError BAD_ARGUMENTS }
SELECT count() > 0 FROM system.zookeeper WHERE value LIKE '%'; -- { serverError BAD_ARGUMENTS }
SELECT count() > 0 FROM system.zookeeper WHERE path LIKE '/%'; -- { serverError BAD_ARGUMENTS }
SELECT count() > 0 FROM system.zookeeper WHERE path = '/';
SET allow_unrestricted_reads_from_keeper = 'true';
SELECT count() > 0 FROM system.zookeeper;
SELECT count() > 0 FROM system.zookeeper WHERE name LIKE '%_%';
SELECT count() > 0 FROM system.zookeeper WHERE value LIKE '%';
SELECT count() > 0 FROM system.zookeeper WHERE path LIKE '/%';
SELECT count() > 0 FROM system.zookeeper WHERE path = '/';

View File

@ -0,0 +1 @@
1 1 1 1 1

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q """
CREATE TABLE t02982
(
n UInt64,
s Nullable(String),
INDEX idx1 n TYPE minmax GRANULARITY 2,
INDEX idx2 n * length(s) TYPE set(1000) GRANULARITY 2,
PROJECTION pr_sort
(
SELECT
n,
sum(length(s))
GROUP BY n
)
)
ENGINE = MergeTree
ORDER BY n;
"""
query_id=$RANDOM
$CLICKHOUSE_CLIENT --query_id $query_id -q """
INSERT INTO t02982 SELECT
number,
'a'
FROM numbers_mt(1000000);
"""
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT -q """
SELECT
ProfileEvents['MergeTreeDataProjectionWriterMergingBlocksMicroseconds'] > 0,
ProfileEvents['MergeTreeDataProjectionWriterSortingBlocksMicroseconds'] > 0,
ProfileEvents['MergeTreeDataWriterSortingBlocksMicroseconds'] > 0,
ProfileEvents['MergeTreeDataWriterProjectionsCalculationMicroseconds'] > 0,
ProfileEvents['MergeTreeDataWriterSkipIndicesCalculationMicroseconds'] > 0
FROM system.query_log
WHERE current_database = currentDatabase() AND query_id='$query_id' AND type = 'QueryFinish';
"""

View File

@ -0,0 +1,7 @@
1
2
3
4
5
6
7

View File

@ -0,0 +1,26 @@
-- Tags: distributed, no-parallel
DROP DATABASE IF EXISTS shard_0;
DROP DATABASE IF EXISTS shard_1;
DROP TABLE IF EXISTS t_distr;
CREATE DATABASE IF NOT EXISTS shard_0;
CREATE DATABASE IF NOT EXISTS shard_1;
CREATE TABLE shard_0.t_local (a Int) ENGINE = Memory;
CREATE TABLE shard_1.t_local (a Int) ENGINE = Memory;
CREATE TABLE t_distr (a Int) ENGINE = Distributed(test_cluster_two_shards_different_databases, '', 't_local', 1000);
SET distributed_foreground_insert=0;
INSERT INTO t_distr VALUES (1), (2), (3);
SET distributed_foreground_insert=1;
INSERT INTO t_distr VALUES (4), (5), (6), (7);
SYSTEM FLUSH DISTRIBUTED t_distr;
SELECT * FROM t_distr ORDER BY a;
DROP TABLE t_distr;
DROP DATABASE shard_0;
DROP DATABASE shard_1;

View File

@ -0,0 +1,7 @@
-- { echoOn }
SELECT f1, f2['2'], count() FROM t1 GROUP BY 1,2 order by 1,2;
1 1000111
SELECT f1, f3['2'], count() FROM t1 GROUP BY 1,2 order by 1,2;
1 1000111
SELECT f1, f4[2], count() FROM t1 GROUP BY 1,2 order by 1,2;
1 0 1000111

View File

@ -0,0 +1,21 @@
--https://github.com/ClickHouse/ClickHouse/issues/59402
CREATE TABLE t1
(
f1 Int32,
f2 Map(LowCardinality(String),LowCardinality(String)),
f3 Map(String,String),
f4 Map(Int32,Int32)
)
ENGINE=Memory AS
SELECT 1 as f1,
map(number%2,number%10) as f2,
f2 as f3,
f2 as f4
from numbers(1000111);
SET max_block_size=10;
-- { echoOn }
SELECT f1, f2['2'], count() FROM t1 GROUP BY 1,2 order by 1,2;
SELECT f1, f3['2'], count() FROM t1 GROUP BY 1,2 order by 1,2;
SELECT f1, f4[2], count() FROM t1 GROUP BY 1,2 order by 1,2;

View File

@ -0,0 +1 @@
[]

View File

@ -0,0 +1,2 @@
-- https://github.com/ClickHouse/ClickHouse/issues/59107
SELECT topK('102.4') FROM remote('127.0.0.{1,2}', view(SELECT NULL FROM system.one WHERE dummy = 1));

View File

@ -322,6 +322,8 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress
return 1;
}
if (0 != munmap(output, le64toh(file_info.uncompressed_size)))
perror("munmap");
if (0 != fsync(output_fd))
perror("fsync");
if (0 != close(output_fd))
@ -528,10 +530,10 @@ int main(int/* argc*/, char* argv[])
(void)snprintf(decompressed_name, decompressed_name_len + 1, decompressed_name_fmt, self, decompressed_suffix);
std::error_code ec;
std::filesystem::copy_file(static_cast<char *>(decompressed_name), static_cast<char *>(self), ec);
if (ec)
if (link(decompressed_name, self))
{
std::cerr << ec.message() << std::endl;
perror("link");
return 1;
}