mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' into allow-parallel-replicas-for-join-with-analyzer-2
This commit is contained in:
commit
c434748fc1
6
.github/ISSUE_TEMPLATE/85_bug-report.md
vendored
6
.github/ISSUE_TEMPLATE/85_bug-report.md
vendored
@ -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**
|
||||
|
||||
|
10
.github/workflows/master.yml
vendored
10
.github/workflows/master.yml
vendored
@ -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
|
||||
|
2
.github/workflows/nightly.yml
vendored
2
.github/workflows/nightly.yml
vendored
@ -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"
|
||||
|
16
.github/workflows/reusable_docker.yml
vendored
16
.github/workflows/reusable_docker.yml
vendored
@ -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
|
||||
|
@ -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)).
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -17,6 +17,7 @@ set (SRCS
|
||||
getMemoryAmount.cpp
|
||||
getPageSize.cpp
|
||||
getThreadId.cpp
|
||||
int8_to_string.cpp
|
||||
JSON.cpp
|
||||
mremap.cpp
|
||||
phdr_cache.cpp
|
||||
|
@ -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;
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
9
base/base/int8_to_string.cpp
Normal file
9
base/base/int8_to_string.cpp
Normal 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});
|
||||
}
|
||||
}
|
17
base/base/int8_to_string.h
Normal file
17
base/base/int8_to_string.h
Normal 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)
|
||||
}
|
@ -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;
|
||||
|
@ -1,5 +1,5 @@
|
||||
---
|
||||
slug: /en/sql-reference/data-types/json
|
||||
slug: /en/sql-reference/data-types/variant
|
||||
sidebar_position: 55
|
||||
sidebar_label: Variant
|
||||
---
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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>
|
||||
|
@ -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(¤t_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(¤t_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(¤t_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;
|
||||
};
|
||||
|
@ -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.") \
|
||||
|
@ -206,6 +206,9 @@ public:
|
||||
*/
|
||||
void merge(const Self & rhs)
|
||||
{
|
||||
if (!rhs.size())
|
||||
return;
|
||||
|
||||
UInt64 m1 = 0;
|
||||
UInt64 m2 = 0;
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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; };
|
||||
|
@ -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) \
|
||||
|
@ -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."},
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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(
|
||||
|
@ -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;
|
||||
|
@ -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) \
|
||||
|
@ -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)));
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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,
|
||||
|
111
tests/ci/ci.py
111
tests/ci/ci.py
@ -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:
|
||||
|
@ -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),
|
||||
|
@ -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:
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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,
|
||||
|
@ -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:
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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")]
|
||||
|
||||
|
@ -43,6 +43,7 @@ TRUSTED_CONTRIBUTORS = {
|
||||
"tsolodov", # ClickHouse, Inc
|
||||
"kitaisreal",
|
||||
"k-morozov", # Konstantin Morozov, Yandex Cloud
|
||||
"justindeguzman", # ClickHouse, Inc
|
||||
]
|
||||
}
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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(
|
||||
|
@ -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(
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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:
|
||||
|
@ -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
|
||||
|
@ -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")]
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -0,0 +1,6 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
17
tests/queries/0_stateless/02976_system_zookeeper_filters.sql
Normal file
17
tests/queries/0_stateless/02976_system_zookeeper_filters.sql
Normal 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 = '/';
|
@ -0,0 +1 @@
|
||||
1 1 1 1 1
|
46
tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh
Executable file
46
tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh
Executable 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';
|
||||
"""
|
@ -0,0 +1,7 @@
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
26
tests/queries/0_stateless/02983_const_sharding_key.sql
Normal file
26
tests/queries/0_stateless/02983_const_sharding_key.sql
Normal 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;
|
7
tests/queries/0_stateless/02983_empty_map.reference
Normal file
7
tests/queries/0_stateless/02983_empty_map.reference
Normal 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
|
21
tests/queries/0_stateless/02983_empty_map.sql
Normal file
21
tests/queries/0_stateless/02983_empty_map.sql
Normal 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;
|
@ -0,0 +1 @@
|
||||
[]
|
2
tests/queries/0_stateless/02984_topk_empty_merge.sql
Normal file
2
tests/queries/0_stateless/02984_topk_empty_merge.sql
Normal 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));
|
@ -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;
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user