Merge branch 'master' into signal_handlers_in_client

This commit is contained in:
Alexey Milovidov 2024-07-14 00:54:30 +02:00 committed by GitHub
commit ce37be3ab1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
74 changed files with 2212 additions and 599 deletions

View File

@ -7,3 +7,4 @@ self-hosted-runner:
- stress-tester
- style-checker
- style-checker-aarch64
- release-maker

View File

@ -6,8 +6,8 @@ concurrency:
'on':
workflow_dispatch:
inputs:
sha:
description: 'The SHA hash of the commit from which to create the release'
ref:
description: 'Git reference (branch or commit sha) from which to create the release'
required: true
type: string
type:
@ -15,15 +15,152 @@ concurrency:
required: true
type: choice
options:
- new
- patch
- new
dry-run:
description: 'Dry run'
required: false
default: true
type: boolean
jobs:
Release:
runs-on: [self-hosted, style-checker-aarch64]
CreateRelease:
env:
GH_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }}
runs-on: [self-hosted, release-maker]
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Set envs
# https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings
run: |
cat >> "$GITHUB_ENV" << 'EOF'
ROBOT_CLICKHOUSE_SSH_KEY<<RCSK
${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}}
RCSK
RELEASE_INFO_FILE=${{ runner.temp }}/release_info.json
EOF
- name: Check out repository code
uses: ClickHouse/checkout@v1
- name: Print greeting
with:
token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}}
fetch-depth: 0
- name: Prepare Release Info
run: |
python3 ./tests/ci/release.py --commit ${{ inputs.sha }} --type ${{ inputs.type }} --dry-run
python3 ./tests/ci/create_release.py --prepare-release-info \
--ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \
--outfile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
echo "::group::Release Info"
python3 -m json.tool "$RELEASE_INFO_FILE"
echo "::endgroup::"
release_tag=$(jq -r '.release_tag' "$RELEASE_INFO_FILE")
commit_sha=$(jq -r '.commit_sha' "$RELEASE_INFO_FILE")
echo "Release Tag: $release_tag"
echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV"
echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV"
- name: Download All Release Artifacts
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/create_release.py --infile "$RELEASE_INFO_FILE" --download-packages ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Push Git Tag for the Release
run: |
python3 ./tests/ci/create_release.py --push-release-tag --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Push New Release Branch
if: ${{ inputs.type == 'new' }}
run: |
python3 ./tests/ci/create_release.py --push-new-release-branch --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Bump CH Version and Update Contributors' List
run: |
python3 ./tests/ci/create_release.py --create-bump-version-pr --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Checkout master
run: |
git checkout master
- name: Bump Docker versions, Changelog, Security
if: ${{ inputs.type == 'patch' }}
run: |
[ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1
echo "List versions"
./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv
echo "Update docker version"
./utils/list-versions/update-docker-version.sh
echo "Generate ChangeLog"
export CI=1
docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \
--volume=".:/ClickHouse" clickhouse/style-test \
/ClickHouse/tests/ci/changelog.py -v --debug-helpers \
--gh-user-or-token="$GH_TOKEN" --jobs=5 \
--output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }}
git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md
echo "Generate Security"
python3 ./utils/security-generator/generate_security.py > SECURITY.md
git diff HEAD
- name: Generate ChangeLog
if: ${{ inputs.type == 'patch' && ! inputs.dry-run }}
uses: peter-evans/create-pull-request@v6
with:
author: "robot-clickhouse <robot-clickhouse@users.noreply.github.com>"
token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }}
committer: "robot-clickhouse <robot-clickhouse@users.noreply.github.com>"
commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }}
branch: auto/${{ env.RELEASE_TAG }}
assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher
delete-branch: true
title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }}
labels: do not test
body: |
Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }}
### Changelog category (leave one):
- Not for changelog (changelog entry is not required)
- name: Reset changes if Dry-run
if: ${{ inputs.dry-run }}
run: |
git reset --hard HEAD
- name: Checkout back to GITHUB_REF
run: |
git checkout "$GITHUB_REF_NAME"
- name: Create GH Release
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/create_release.py --create-gh-release \
--infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Export TGZ Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --export-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Test TGZ Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --test-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Export RPM Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --export-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Test RPM Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --test-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Export Debian Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --export-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Test Debian Packages
if: ${{ inputs.type == 'patch' }}
run: |
python3 ./tests/ci/artifactory.py --test-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }}
- name: Docker clickhouse/clickhouse-server building
if: ${{ inputs.type == 'patch' }}
run: |
cd "./tests/ci"
export CHECK_NAME="Docker server image"
python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }}
- name: Docker clickhouse/clickhouse-keeper building
if: ${{ inputs.type == 'patch' }}
run: |
cd "./tests/ci"
export CHECK_NAME="Docker keeper image"
python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }}
- name: Post Slack Message
if: always()
run: |
echo Slack Message

View File

@ -172,7 +172,7 @@ jobs:
################################# Stage Final #################################
#
FinishCheck:
if: ${{ !failure() }}
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
runs-on: [self-hosted, style-checker-aarch64]
steps:

View File

@ -42,9 +42,19 @@ endif ()
# But use 2 parallel jobs, since:
# - this is what llvm does
# - and I've verfied that lld-11 does not use all available CPU time (in peak) while linking one binary
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLEL_LINK_JOBS GREATER 2)
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
set (PARALLEL_LINK_JOBS 2)
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO)
if (ARCH_AARCH64)
# aarch64 builds start to often fail with OOMs (reason not yet clear), for now let's limit the concurrency
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 1.")
set (PARALLEL_LINK_JOBS 1)
if (LINKER_NAME MATCHES "lld")
math(EXPR LTO_JOBS ${NUMBER_OF_LOGICAL_CORES}/4)
set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -Wl,--thinlto-jobs=${LTO_JOBS}")
endif()
elseif (PARALLEL_LINK_JOBS GREATER 2)
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
set (PARALLEL_LINK_JOBS 2)
endif ()
endif()
message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB RAM, 'OFF' means the native core count).")

View File

@ -27,19 +27,19 @@ def run_fuzzer(fuzzer: str):
parser.read(path)
if parser.has_section("asan"):
os.environ[
"ASAN_OPTIONS"
] = f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}"
os.environ["ASAN_OPTIONS"] = (
f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}"
)
if parser.has_section("msan"):
os.environ[
"MSAN_OPTIONS"
] = f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}"
os.environ["MSAN_OPTIONS"] = (
f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}"
)
if parser.has_section("ubsan"):
os.environ[
"UBSAN_OPTIONS"
] = f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}"
os.environ["UBSAN_OPTIONS"] = (
f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}"
)
if parser.has_section("libfuzzer"):
custom_libfuzzer_options = " ".join(

View File

@ -23,7 +23,10 @@ source /utils.lib
/usr/share/clickhouse-test/config/install.sh
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence &
./setup_minio.sh stateful
./mc admin trace clickminio > /test_output/rubbish.log &
MC_ADMIN_PID=$!
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
@ -254,6 +257,8 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
sudo clickhouse stop --pid-path /var/run/clickhouse-server2 ||:
fi
# Kill minio admin client to stop collecting logs
kill $MC_ADMIN_PID
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst ||:

View File

@ -86,6 +86,7 @@ RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoo
ENV MINIO_ROOT_USER="clickhouse"
ENV MINIO_ROOT_PASSWORD="clickhouse"
ENV EXPORT_S3_STORAGE_POLICIES=1
ENV CLICKHOUSE_GRPC_CLIENT="/usr/share/clickhouse-utils/grpc-client/clickhouse-grpc-client.py"
RUN npm install -g azurite@3.30.0 \
&& npm install -g tslib && npm install -g node

View File

@ -8,6 +8,7 @@ cryptography==3.4.8
dbus-python==1.2.18
distro==1.7.0
docutils==0.17.1
grpcio==1.47.0
gyp==0.1
httplib2==0.20.2
idna==3.3
@ -28,6 +29,7 @@ packaging==24.1
pandas==1.5.3
pip==24.1.1
pipdeptree==2.23.0
protobuf==4.25.3
pyarrow==15.0.0
pyasn1==0.4.8
PyJWT==2.3.0

View File

@ -6,8 +6,8 @@ source /setup_export_logs.sh
# fail on errors, verbose and export all env variables
set -e -x -a
MAX_RUN_TIME=${MAX_RUN_TIME:-10800}
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME))
MAX_RUN_TIME=${MAX_RUN_TIME:-7200}
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
@ -54,6 +54,9 @@ source /utils.lib
/usr/share/clickhouse-test/config/install.sh
./setup_minio.sh stateless
m./c admin trace clickminio > /test_output/rubbish.log &
MC_ADMIN_PID=$!
./setup_hdfs_minicluster.sh
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
@ -320,7 +323,7 @@ export -f run_tests
# This should be enough to setup job and collect artifacts
TIMEOUT=$((MAX_RUN_TIME - 600))
TIMEOUT=$((MAX_RUN_TIME - 700))
if [ "$NUM_TRIES" -gt "1" ]; then
# We don't run tests with Ordinary database in PRs, only in master.
# So run new/changed tests with Ordinary at least once in flaky check.
@ -383,6 +386,9 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||:
fi
# Kill minio admin client to stop collecting logs
kill $MC_ADMIN_PID
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server.log ||:
rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst &

View File

@ -3,7 +3,7 @@ aiosignal==1.3.1
astroid==3.1.0
async-timeout==4.0.3
attrs==23.2.0
black==23.12.0
black==24.4.2
boto3==1.34.131
botocore==1.34.131
certifi==2024.6.2

View File

@ -11,6 +11,7 @@ TIMEOUT_SIGN = "[ Timeout! "
UNKNOWN_SIGN = "[ UNKNOWN "
SKIPPED_SIGN = "[ SKIPPED "
HUNG_SIGN = "Found hung queries in processlist"
SERVER_DIED_SIGN = "Server died, terminating all processes"
DATABASE_SIGN = "Database: "
SUCCESS_FINISH_SIGNS = ["All tests have finished", "No tests were run"]
@ -25,6 +26,7 @@ def process_test_log(log_path, broken_tests):
failed = 0
success = 0
hung = False
server_died = False
retries = False
success_finish = False
test_results = []
@ -41,6 +43,8 @@ def process_test_log(log_path, broken_tests):
if HUNG_SIGN in line:
hung = True
break
if SERVER_DIED_SIGN in line:
server_died = True
if RETRIES_SIGN in line:
retries = True
if any(
@ -123,6 +127,7 @@ def process_test_log(log_path, broken_tests):
failed,
success,
hung,
server_died,
success_finish,
retries,
test_results,
@ -150,6 +155,7 @@ def process_result(result_path, broken_tests):
failed,
success,
hung,
server_died,
success_finish,
retries,
test_results,
@ -165,6 +171,10 @@ def process_result(result_path, broken_tests):
description = "Some queries hung, "
state = "failure"
test_results.append(("Some queries hung", "FAIL", "0", ""))
elif server_died:
description = "Server died, "
state = "failure"
test_results.append(("Server died", "FAIL", "0", ""))
elif not success_finish:
description = "Tests are not finished, "
state = "failure"
@ -218,5 +228,20 @@ if __name__ == "__main__":
state, description, test_results = process_result(args.in_results_dir, broken_tests)
logging.info("Result parsed")
status = (state, description)
def test_result_comparator(item):
# sort by status then by check name
order = {
"FAIL": 0,
"Timeout": 1,
"NOT_FAILED": 2,
"BROKEN": 3,
"OK": 4,
"SKIPPED": 5,
}
return order.get(item[1], 10), str(item[0]), item[1]
test_results.sort(key=test_result_comparator)
write_results(args.out_results_file, args.out_status_file, test_results, status)
logging.info("Result written")

View File

@ -376,6 +376,7 @@ void LocalServer::setupUsers()
" </networks>"
" <profile>default</profile>"
" <quota>default</quota>"
" <named_collection_control>1</named_collection_control>"
" </default>"
" </users>"
" <quotas>"

View File

@ -516,6 +516,9 @@
/// Save query in history only if it is different.
let previous_query = '';
/// Start of the last query
let last_query_start = 0;
const current_url = new URL(window.location);
const opened_locally = location.protocol == 'file:';
@ -567,6 +570,8 @@
'&password=' + encodeURIComponent(password)
}
last_query_start = performance.now();
const xhr = new XMLHttpRequest;
xhr.open('POST', url, true);
@ -579,7 +584,8 @@
if (posted_request_num != request_num) {
return;
} else if (this.readyState === XMLHttpRequest.DONE) {
renderResponse(this.status, this.response);
const elapsed_msec = performance.now() - last_query_start;
renderResponse(this.status, this.response, elapsed_msec);
/// The query is saved in browser history (in state JSON object)
/// as well as in URL fragment identifier.
@ -587,7 +593,8 @@
const state = {
query: query,
status: this.status,
response: this.response.length > 100000 ? null : this.response /// Lower than the browser's limit.
response: this.response.length > 100000 ? null : this.response, /// Lower than the browser's limit.
elapsed_msec: elapsed_msec,
};
const title = "ClickHouse Query: " + query;
@ -617,7 +624,7 @@
xhr.send(query);
}
function renderResponse(status, response) {
function renderResponse(status, response, elapsed_msec) {
document.getElementById('hourglass').style.display = 'none';
if (status === 200) {
@ -632,6 +639,7 @@
renderChart(json);
} else {
renderUnparsedResult(response);
stats.innerText = `Elapsed (client-side): ${(elapsed_msec / 1000).toFixed(3)} sec.`;
}
document.getElementById('check-mark').style.display = 'inline';
} else {
@ -651,7 +659,7 @@
clear();
return;
}
renderResponse(event.state.status, event.state.response);
renderResponse(event.state.status, event.state.response, event.state.elapsed_msec);
};
if (window.location.hash) {

View File

@ -17,6 +17,8 @@ src_paths = ["src", "tests/ci", "tests/sqllogic"]
[tool.pylint.'MESSAGES CONTROL']
# pytest.mark.parametrize is not callable (not-callable)
disable = '''
pointless-string-statement,
line-too-long,
missing-docstring,
too-few-public-methods,
invalid-name,
@ -36,6 +38,7 @@ disable = '''
bare-except,
no-else-return,
global-statement,
f-string-without-interpolation,
'''
[tool.pylint.SIMILARITIES]

View File

@ -996,6 +996,10 @@ void ZooKeeper::receiveEvent()
if (request_info.callback)
request_info.callback(*response);
/// Finalize current session if we receive a hardware error from ZooKeeper
if (err != Error::ZOK && isHardwareError(err))
finalize(/*error_send*/ false, /*error_receive*/ true, fmt::format("Hardware error: {}", err));
}

View File

@ -25,7 +25,7 @@ namespace DB
template <typename To, typename From>
inline To assert_cast(From && from)
{
#ifndef NDEBUG
#ifdef ABORT_ON_LOGICAL_ERROR
try
{
if constexpr (std::is_pointer_v<To>)

View File

@ -49,7 +49,7 @@ namespace
const String & dest_blob_,
std::shared_ptr<const AzureBlobStorage::RequestSettings> settings_,
ThreadPoolCallbackRunnerUnsafe<void> schedule_,
const Poco::Logger * log_)
LoggerPtr log_)
: create_read_buffer(create_read_buffer_)
, client(client_)
, offset (offset_)
@ -74,7 +74,7 @@ namespace
const String & dest_blob;
std::shared_ptr<const AzureBlobStorage::RequestSettings> settings;
ThreadPoolCallbackRunnerUnsafe<void> schedule;
const Poco::Logger * log;
const LoggerPtr log;
size_t max_single_part_upload_size;
struct UploadPartTask
@ -83,7 +83,6 @@ namespace
size_t part_size;
std::vector<std::string> block_ids;
bool is_finished = false;
std::exception_ptr exception;
};
size_t normal_part_size;
@ -92,6 +91,7 @@ namespace
std::list<UploadPartTask> TSA_GUARDED_BY(bg_tasks_mutex) bg_tasks;
int num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
int num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
std::exception_ptr bg_exception TSA_GUARDED_BY(bg_tasks_mutex);
std::mutex bg_tasks_mutex;
std::condition_variable bg_tasks_condvar;
@ -186,7 +186,7 @@ namespace
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
tryLogCurrentException(log, fmt::format("While performing multipart upload of blob {} in container {}", dest_blob, dest_container_for_logging));
waitForAllBackgroundTasks();
throw;
}
@ -242,7 +242,12 @@ namespace
}
catch (...)
{
task->exception = std::current_exception();
std::lock_guard lock(bg_tasks_mutex);
if (!bg_exception)
{
tryLogCurrentException(log, "While writing part");
bg_exception = std::current_exception(); /// The exception will be rethrown after all background tasks stop working.
}
}
task_finish_notify();
}, Priority{});
@ -299,13 +304,13 @@ namespace
/// Suppress warnings because bg_tasks_mutex is actually hold, but tsa annotations do not understand std::unique_lock
bg_tasks_condvar.wait(lock, [this]() {return TSA_SUPPRESS_WARNING_FOR_READ(num_added_bg_tasks) == TSA_SUPPRESS_WARNING_FOR_READ(num_finished_bg_tasks); });
auto & tasks = TSA_SUPPRESS_WARNING_FOR_WRITE(bg_tasks);
for (auto & task : tasks)
{
if (task.exception)
std::rethrow_exception(task.exception);
auto exception = TSA_SUPPRESS_WARNING_FOR_READ(bg_exception);
if (exception)
std::rethrow_exception(exception);
const auto & tasks = TSA_SUPPRESS_WARNING_FOR_READ(bg_tasks);
for (const auto & task : tasks)
block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end());
}
}
};
}
@ -321,7 +326,8 @@ void copyDataToAzureBlobStorageFile(
std::shared_ptr<const AzureBlobStorage::RequestSettings> settings,
ThreadPoolCallbackRunnerUnsafe<void> schedule)
{
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, &Poco::Logger::get("copyDataToAzureBlobStorageFile")};
auto log = getLogger("copyDataToAzureBlobStorageFile");
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, log};
helper.performCopy();
}
@ -339,9 +345,11 @@ void copyAzureBlobStorageFile(
const ReadSettings & read_settings,
ThreadPoolCallbackRunnerUnsafe<void> schedule)
{
auto log = getLogger("copyAzureBlobStorageFile");
if (settings->use_native_copy)
{
LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob);
LOG_TRACE(log, "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob);
ProfileEvents::increment(ProfileEvents::AzureCopyObject);
if (dest_client->GetClickhouseOptions().IsClientForDisk)
ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject);
@ -352,7 +360,7 @@ void copyAzureBlobStorageFile(
if (size < settings->max_single_part_copy_size)
{
LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy blob sync {} -> {}", src_blob, dest_blob);
LOG_TRACE(log, "Copy blob sync {} -> {}", src_blob, dest_blob);
block_blob_client_dest.CopyFromUri(source_uri);
}
else
@ -368,7 +376,7 @@ void copyAzureBlobStorageFile(
if (copy_status.HasValue() && copy_status.Value() == Azure::Storage::Blobs::Models::CopyStatus::Success)
{
LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy of {} to {} finished", properties_model.CopySource.Value(), dest_blob);
LOG_TRACE(log, "Copy of {} to {} finished", properties_model.CopySource.Value(), dest_blob);
}
else
{
@ -382,14 +390,14 @@ void copyAzureBlobStorageFile(
}
else
{
LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob);
LOG_TRACE(log, "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob);
auto create_read_buffer = [&]
{
return std::make_unique<ReadBufferFromAzureBlobStorage>(
src_client, src_blob, read_settings, settings->max_single_read_retries, settings->max_single_download_retries);
};
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, &Poco::Logger::get("copyAzureBlobStorageFile")};
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, log};
helper.performCopy();
}
}

View File

@ -98,7 +98,6 @@ namespace
size_t part_size;
String tag;
bool is_finished = false;
std::exception_ptr exception;
};
size_t num_parts;
@ -111,6 +110,7 @@ namespace
size_t num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
size_t num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
size_t num_finished_parts TSA_GUARDED_BY(bg_tasks_mutex) = 0;
std::exception_ptr bg_exception TSA_GUARDED_BY(bg_tasks_mutex);
std::mutex bg_tasks_mutex;
std::condition_variable bg_tasks_condvar;
@ -273,7 +273,7 @@ namespace
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
tryLogCurrentException(log, fmt::format("While performing multipart upload of {}", dest_key));
// Multipart upload failed because it wasn't possible to schedule all the tasks.
// To avoid execution of already scheduled tasks we abort MultipartUpload.
abortMultipartUpload();
@ -385,7 +385,12 @@ namespace
}
catch (...)
{
task->exception = std::current_exception();
std::lock_guard lock(bg_tasks_mutex);
if (!bg_exception)
{
tryLogCurrentException(log, fmt::format("While writing part #{}", task->part_number));
bg_exception = std::current_exception(); /// The exception will be rethrown after all background tasks stop working.
}
}
task_finish_notify();
}, Priority{});
@ -435,22 +440,21 @@ namespace
/// Suppress warnings because bg_tasks_mutex is actually hold, but tsa annotations do not understand std::unique_lock
bg_tasks_condvar.wait(lock, [this]() {return TSA_SUPPRESS_WARNING_FOR_READ(num_added_bg_tasks) == TSA_SUPPRESS_WARNING_FOR_READ(num_finished_bg_tasks); });
auto & tasks = TSA_SUPPRESS_WARNING_FOR_WRITE(bg_tasks);
for (auto & task : tasks)
auto exception = TSA_SUPPRESS_WARNING_FOR_READ(bg_exception);
if (exception)
{
if (task.exception)
{
/// abortMultipartUpload() might be called already, see processUploadPartRequest().
/// However if there were concurrent uploads at that time, those part uploads might or might not succeed.
/// As a result, it might be necessary to abort a given multipart upload multiple times in order to completely free
/// all storage consumed by all parts.
abortMultipartUpload();
/// abortMultipartUpload() might be called already, see processUploadPartRequest().
/// However if there were concurrent uploads at that time, those part uploads might or might not succeed.
/// As a result, it might be necessary to abort a given multipart upload multiple times in order to completely free
/// all storage consumed by all parts.
abortMultipartUpload();
std::rethrow_exception(task.exception);
}
part_tags.push_back(task.tag);
std::rethrow_exception(exception);
}
const auto & tasks = TSA_SUPPRESS_WARNING_FOR_READ(bg_tasks);
for (const auto & task : tasks)
part_tags.push_back(task.tag);
}
};

View File

@ -1281,10 +1281,6 @@ void DatabaseCatalog::rescheduleDropTableTask()
auto min_drop_time = getMinDropTime();
time_t schedule_after_ms = min_drop_time > current_time ? (min_drop_time - current_time) * 1000 : 0;
LOG_TRACE(
log,
"Have {} tables in queue to drop. Schedule background task in {} seconds",
tables_marked_dropped.size(), schedule_after_ms / 1000);
(*drop_task)->scheduleAfter(schedule_after_ms);
}

View File

@ -445,6 +445,9 @@ bool NpyRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & /*
elements_in_current_column *= header.shape[i];
}
if (typeid_cast<ColumnArray *>(current_column))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected nesting level of column '{}', expected {}", column->getName(), header.shape.size() - 1);
for (size_t i = 0; i != elements_in_current_column; ++i)
readValue(current_column);

View File

@ -196,6 +196,16 @@ void DistributedAsyncInsertBatch::readText(ReadBuffer & in)
UInt64 idx;
in >> idx >> "\n";
files.push_back(std::filesystem::absolute(fmt::format("{}/{}.bin", parent.path, idx)).string());
ReadBufferFromFile header_buffer(files.back());
const DistributedAsyncInsertHeader & header = DistributedAsyncInsertHeader::read(header_buffer, parent.log);
total_bytes += total_bytes;
if (header.rows)
{
total_rows += header.rows;
total_bytes += header.bytes;
}
}
recovered = true;

View File

@ -101,9 +101,8 @@ struct MergeTreePartInfo
bool isFakeDropRangePart() const
{
/// Another max level was previously used for REPLACE/MOVE PARTITION
auto another_max_level = std::numeric_limits<decltype(level)>::max();
return level == MergeTreePartInfo::MAX_LEVEL || level == another_max_level;
/// LEGACY_MAX_LEVEL was previously used for REPLACE/MOVE PARTITION
return level == MergeTreePartInfo::MAX_LEVEL || level == MergeTreePartInfo::LEGACY_MAX_LEVEL;
}
String getPartNameAndCheckFormat(MergeTreeDataFormatVersion format_version) const;

355
tests/ci/artifactory.py Normal file
View File

@ -0,0 +1,355 @@
import argparse
import time
from pathlib import Path
from typing import Optional
from shutil import copy2
from create_release import PackageDownloader, ReleaseInfo, ShellRunner
from ci_utils import WithIter
class MountPointApp(metaclass=WithIter):
RCLONE = "rclone"
S3FS = "s3fs"
class R2MountPoint:
_TEST_BUCKET_NAME = "repo-test"
_PROD_BUCKET_NAME = "packages"
_CACHE_MAX_SIZE_GB = 20
MOUNT_POINT = "/home/ubuntu/mountpoint"
API_ENDPOINT = "https://d4fd593eebab2e3a58a599400c4cd64d.r2.cloudflarestorage.com"
LOG_FILE = "/home/ubuntu/fuse_mount.log"
# mod time is not required by reprepro and createrepo - disable to simplify bucket's mount sync (applicable fro rclone)
NOMODTIME = True
# enable debug messages in mount log
DEBUG = True
# enable cache for mountpoint
CACHE_ENABLED = False
# TODO: which mode is better: minimal/writes/full/off
_RCLONE_CACHE_MODE = "minimal"
UMASK = "0000"
def __init__(self, app: str, dry_run: bool) -> None:
assert app in MountPointApp
self.app = app
if dry_run:
self.bucket_name = self._TEST_BUCKET_NAME
else:
self.bucket_name = self._PROD_BUCKET_NAME
self.aux_mount_options = ""
self.async_mount = False
if self.app == MountPointApp.S3FS:
self.cache_dir = "/home/ubuntu/s3fs_cache"
# self.aux_mount_options += "-o nomodtime " if self.NOMODTIME else "" not for s3fs
self.aux_mount_options += "--debug " if self.DEBUG else ""
self.aux_mount_options += (
f"-o use_cache={self.cache_dir} -o cache_size_mb={self._CACHE_MAX_SIZE_GB * 1024} "
if self.CACHE_ENABLED
else ""
)
# without -o nomultipart there are errors like "Error 5 writing to /home/ubuntu/***.deb: Input/output error"
self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}"
elif self.app == MountPointApp.RCLONE:
# run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return
self.async_mount = True
self.cache_dir = "/home/ubuntu/rclone_cache"
self.aux_mount_options += "--no-modtime " if self.NOMODTIME else ""
self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose
self.aux_mount_options += (
f"--vfs-cache-mode {self._RCLONE_CACHE_MODE} --vfs-cache-max-size {self._CACHE_MAX_SIZE_GB}G"
if self.CACHE_ENABLED
else "--vfs-cache-mode off"
)
# Use --no-modtime to try to avoid: ERROR : rpm/lts/clickhouse-client-24.3.6.5.x86_64.rpm: Failed to apply pending mod time
self.mount_cmd = f"rclone mount remote:{self.bucket_name} {self.MOUNT_POINT} --daemon --cache-dir {self.cache_dir} --umask 0000 --log-file {self.LOG_FILE} {self.aux_mount_options}"
else:
assert False
def init(self):
print(f"Mount bucket [{self.bucket_name}] to [{self.MOUNT_POINT}]")
_CLEAN_LOG_FILE_CMD = f"tail -n 1000 {self.LOG_FILE} > {self.LOG_FILE}_tmp && mv {self.LOG_FILE}_tmp {self.LOG_FILE} ||:"
_MKDIR_CMD = f"mkdir -p {self.MOUNT_POINT}"
_MKDIR_FOR_CACHE = f"mkdir -p {self.cache_dir}"
_UNMOUNT_CMD = (
f"mount | grep -q {self.MOUNT_POINT} && umount {self.MOUNT_POINT} ||:"
)
_TEST_MOUNT_CMD = f"mount | grep -q {self.MOUNT_POINT}"
ShellRunner.run(_CLEAN_LOG_FILE_CMD)
ShellRunner.run(_UNMOUNT_CMD)
ShellRunner.run(_MKDIR_CMD)
ShellRunner.run(_MKDIR_FOR_CACHE)
ShellRunner.run(self.mount_cmd, async_=self.async_mount)
if self.async_mount:
time.sleep(3)
ShellRunner.run(_TEST_MOUNT_CMD)
@classmethod
def teardown(cls):
print(f"Unmount [{cls.MOUNT_POINT}]")
ShellRunner.run(f"umount {cls.MOUNT_POINT}")
class RepoCodenames(metaclass=WithIter):
LTS = "lts"
STABLE = "stable"
class DebianArtifactory:
_TEST_REPO_URL = "https://pub-73dd1910f4284a81a02a67018967e028.r2.dev/deb"
_PROD_REPO_URL = "https://packages.clickhouse.com/deb"
def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.codename = release_info.codename
self.version = release_info.version
if dry_run:
self.repo_url = self._TEST_REPO_URL
else:
self.repo_url = self._PROD_REPO_URL
assert self.codename in RepoCodenames
self.pd = PackageDownloader(
release=release_info.release_branch,
commit_sha=release_info.commit_sha,
version=release_info.version,
)
def export_packages(self):
assert self.pd.local_deb_packages_ready(), "BUG: Packages are not downloaded"
print("Start adding packages")
paths = [
self.pd.LOCAL_DIR + "/" + file for file in self.pd.get_deb_packages_files()
]
REPREPRO_CMD_PREFIX = f"reprepro --basedir {R2MountPoint.MOUNT_POINT}/configs/deb --outdir {R2MountPoint.MOUNT_POINT}/deb --verbose"
cmd = f"{REPREPRO_CMD_PREFIX} includedeb {self.codename} {' '.join(paths)}"
print("Running export command:")
print(f" {cmd}")
ShellRunner.run(cmd)
ShellRunner.run("sync")
if self.codename == RepoCodenames.LTS:
packages_with_version = [
package + "=" + self.version for package in self.pd.get_packages_names()
]
print(
f"Copy packages from {RepoCodenames.LTS} to {RepoCodenames.STABLE} repository"
)
cmd = f"{REPREPRO_CMD_PREFIX} copy {RepoCodenames.STABLE} {RepoCodenames.LTS} {' '.join(packages_with_version)}"
print("Running copy command:")
print(f" {cmd}")
ShellRunner.run(cmd)
ShellRunner.run("sync")
def test_packages(self):
ShellRunner.run("docker pull ubuntu:latest")
print(f"Test packages installation, version [{self.version}]")
cmd = f"docker run --rm ubuntu:latest bash -c \"apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-client={self.version}\""
print("Running test command:")
print(f" {cmd}")
ShellRunner.run(cmd)
def _copy_if_not_exists(src: Path, dst: Path) -> Path:
if dst.is_dir():
dst = dst / src.name
if not dst.exists():
return copy2(src, dst) # type: ignore
if src.stat().st_size == dst.stat().st_size:
return dst
return copy2(src, dst) # type: ignore
class RpmArtifactory:
_TEST_REPO_URL = (
"https://pub-73dd1910f4284a81a02a67018967e028.r2.dev/rpm/clickhouse.repo"
)
_PROD_REPO_URL = "https://packages.clickhouse.com/rpm/clickhouse.repo"
_SIGN_KEY = "885E2BDCF96B0B45ABF058453E4AD4719DDE9A38"
def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.codename = release_info.codename
self.version = release_info.version
if dry_run:
self.repo_url = self._TEST_REPO_URL
else:
self.repo_url = self._PROD_REPO_URL
assert self.codename in RepoCodenames
self.pd = PackageDownloader(
release=release_info.release_branch,
commit_sha=release_info.commit_sha,
version=release_info.version,
)
def export_packages(self, codename: Optional[str] = None) -> None:
assert self.pd.local_rpm_packages_ready(), "BUG: Packages are not downloaded"
codename = codename or self.codename
print(f"Start adding packages to [{codename}]")
paths = [
self.pd.LOCAL_DIR + "/" + file for file in self.pd.get_rpm_packages_files()
]
dest_dir = Path(R2MountPoint.MOUNT_POINT) / "rpm" / codename
for package in paths:
_copy_if_not_exists(Path(package), dest_dir)
commands = (
f"createrepo_c --local-sqlite --workers=2 --update --verbose {dest_dir}",
f"gpg --sign-with {self._SIGN_KEY} --detach-sign --batch --yes --armor {dest_dir / 'repodata' / 'repomd.xml'}",
)
print(f"Exporting RPM packages into [{codename}]")
for command in commands:
print("Running command:")
print(f" {command}")
ShellRunner.run(command)
update_public_key = f"gpg --armor --export {self._SIGN_KEY}"
pub_key_path = dest_dir / "repodata" / "repomd.xml.key"
print("Updating repomd.xml.key")
pub_key_path.write_text(ShellRunner.run(update_public_key)[1])
if codename == RepoCodenames.LTS:
self.export_packages(RepoCodenames.STABLE)
ShellRunner.run("sync")
def test_packages(self):
ShellRunner.run("docker pull fedora:latest")
print(f"Test package installation, version [{self.version}]")
cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1"'
print("Running test command:")
print(f" {cmd}")
ShellRunner.run(cmd)
class TgzArtifactory:
_TEST_REPO_URL = "https://pub-73dd1910f4284a81a02a67018967e028.r2.dev/tgz"
_PROD_REPO_URL = "https://packages.clickhouse.com/tgz"
def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.codename = release_info.codename
self.version = release_info.version
if dry_run:
self.repo_url = self._TEST_REPO_URL
else:
self.repo_url = self._PROD_REPO_URL
assert self.codename in RepoCodenames
self.pd = PackageDownloader(
release=release_info.release_branch,
commit_sha=release_info.commit_sha,
version=release_info.version,
)
def export_packages(self, codename: Optional[str] = None) -> None:
assert self.pd.local_tgz_packages_ready(), "BUG: Packages are not downloaded"
codename = codename or self.codename
paths = [
self.pd.LOCAL_DIR + "/" + file for file in self.pd.get_tgz_packages_files()
]
dest_dir = Path(R2MountPoint.MOUNT_POINT) / "tgz" / codename
print(f"Exporting TGZ packages into [{codename}]")
for package in paths:
_copy_if_not_exists(Path(package), dest_dir)
if codename == RepoCodenames.LTS:
self.export_packages(RepoCodenames.STABLE)
ShellRunner.run("sync")
def test_packages(self):
tgz_file = "/tmp/tmp.tgz"
tgz_sha_file = "/tmp/tmp.tgz.sha512"
ShellRunner.run(
f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz"
)
ShellRunner.run(
f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512"
)
expected_checksum = ShellRunner.run(f"cut -d ' ' -f 1 {tgz_sha_file}")
actual_checksum = ShellRunner.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1")
assert (
expected_checksum == actual_checksum
), f"[{actual_checksum} != {expected_checksum}]"
ShellRunner.run("rm /tmp/tmp.tgz*")
def parse_args() -> argparse.Namespace:
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Adds release packages to the repository",
)
parser.add_argument(
"--infile",
type=str,
required=True,
help="input file with release info",
)
parser.add_argument(
"--export-debian",
action="store_true",
help="Export debian packages to repository",
)
parser.add_argument(
"--export-rpm",
action="store_true",
help="Export rpm packages to repository",
)
parser.add_argument(
"--export-tgz",
action="store_true",
help="Export tgz packages to repository",
)
parser.add_argument(
"--test-debian",
action="store_true",
help="Test debian packages installation",
)
parser.add_argument(
"--test-rpm",
action="store_true",
help="Test rpm packages installation",
)
parser.add_argument(
"--test-tgz",
action="store_true",
help="Test tgz packages installation",
)
parser.add_argument(
"--dry-run",
action="store_true",
help="Dry run mode",
)
return parser.parse_args()
if __name__ == "__main__":
args = parse_args()
assert args.dry_run
release_info = ReleaseInfo.from_file(args.infile)
"""
Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve:
ERROR : IO error: NotImplemented: versionId not implemented
Failed to copy: NotImplemented: versionId not implemented
"""
mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run)
if args.export_debian:
mp.init()
DebianArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown()
if args.export_rpm:
mp.init()
RpmArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown()
if args.export_tgz:
mp.init()
TgzArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown()
if args.test_debian:
DebianArtifactory(release_info, dry_run=args.dry_run).test_packages()
if args.test_tgz:
TgzArtifactory(release_info, dry_run=args.dry_run).test_packages()
if args.test_rpm:
RpmArtifactory(release_info, dry_run=args.dry_run).test_packages()

View File

@ -15,7 +15,7 @@ import upload_result_helper
from build_check import get_release_or_pr
from ci_config import CI
from ci_metadata import CiMetadata
from ci_utils import GHActions, normalize_string, Shell
from ci_utils import GHActions, normalize_string, Utils
from clickhouse_helper import (
CiLogsCredentials,
ClickHouseHelper,
@ -264,7 +264,7 @@ def check_missing_images_on_dockerhub(
def _pre_action(s3, indata, pr_info):
print("Clear dmesg")
Shell.run("sudo dmesg --clear ||:")
Utils.clear_dmesg()
CommitStatusData.cleanup()
JobReport.cleanup()
BuildResult.cleanup()
@ -325,8 +325,8 @@ def _mark_success_action(
# do nothing, exit without failure
print(f"ERROR: no status file for job [{job}]")
if job_config.run_always or job_config.run_by_label:
print(f"Job [{job}] runs always or by label in CI - do not cache")
if job_config.run_by_label or not job_config.has_digest():
print(f"Job [{job}] has no digest or run by label in CI - do not cache")
else:
if pr_info.is_master:
pass
@ -995,6 +995,10 @@ def main() -> int:
ci_settings,
args.skip_jobs,
)
if IS_CI and pr_info.is_pr and not ci_settings.no_ci_cache:
ci_cache.filter_out_not_affected_jobs()
ci_cache.print_status()
if IS_CI and not pr_info.is_merge_queue:
@ -1031,6 +1035,7 @@ def main() -> int:
elif args.pre:
assert indata, "Run config must be provided via --infile"
_pre_action(s3, indata, pr_info)
JobReport.create_pre_report().dump()
### RUN action: start
elif args.run:
@ -1082,6 +1087,16 @@ def main() -> int:
print(status)
print("::endgroup::")
previous_status = status.state
print("Create dummy job report with job_skipped flag")
JobReport(
status=status.state,
description="",
test_results=[],
start_time="",
duration=0.0,
additional_files=[],
job_skipped=True,
).dump()
# ci cache check
if not previous_status and not ci_settings.no_ci_cache:
@ -1117,20 +1132,22 @@ def main() -> int:
exit_code = 1
else:
exit_code = _run_test(check_name, args.run_command)
job_report = JobReport.load() if JobReport.exist() else None
assert (
job_report
), "BUG. There must be job report either real report, or pre-report if job was killed"
job_report.exit_code = exit_code
job_report.dump()
### RUN action: end
### POST action: start
elif args.post:
if Shell.check(
"sudo dmesg -T | grep -q -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE'"
):
print("WARNING: OOM while job execution")
CIBuddy(dry_run=not pr_info.is_release).post_error(
"Out Of Memory", job_name=_get_ext_check_name(args.job_name)
)
job_report = JobReport.load() if JobReport.exist() else None
if job_report:
assert (
job_report
), "BUG. There must be job report either real report, or pre-report if job was killed"
if not job_report.job_skipped and not job_report.pre_report:
# it's a real job report
ch_helper = ClickHouseHelper()
check_url = ""
@ -1230,9 +1247,32 @@ def main() -> int:
indata["build"],
ch_helper,
)
else:
# no job report
print(f"No job report for {[args.job_name]} - do nothing")
elif job_report.job_skipped:
print(f"Skipped after rerun check {[args.job_name]} - do nothing")
elif job_report.job_skipped:
print(f"Job was skipped {[args.job_name]} - do nothing")
elif job_report.pre_report:
print(f"ERROR: Job was killed - generate evidence")
job_report.update_duration()
# Job was killed!
if Utils.is_killed_with_oom():
print("WARNING: OOM while job execution")
error = f"Out Of Memory, exit_code {job_report.exit_code}, after {job_report.duration}s"
else:
error = f"Unknown, exit_code {job_report.exit_code}, after {job_report.duration}s"
CIBuddy().post_error(error, job_name=_get_ext_check_name(args.job_name))
if CI.is_test_job(args.job_name):
gh = GitHub(get_best_robot_token(), per_page=100)
commit = get_commit(gh, pr_info.sha)
post_commit_status(
commit,
ERROR,
"",
"Error: " + error,
_get_ext_check_name(args.job_name),
pr_info,
dump_to_file=True,
)
### POST action: end
### MARK SUCCESS action: start

View File

@ -520,6 +520,35 @@ class CiCache:
self.RecordType.SUCCESSFUL, job, batch, num_batches, release_branch
)
def has_evidence(self, job: str, job_config: CI.JobConfig) -> bool:
"""
checks if the job has been seen in master/release CI
function is to be used to check if change did not affect the job
:param job_config:
:param job:
:return:
"""
return (
self.is_successful(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
or self.is_pending(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
or self.is_failed(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
)
def is_failed(
self, job: str, batch: int, num_batches: int, release_branch: bool
) -> bool:
@ -609,7 +638,7 @@ class CiCache:
pushes pending records for all jobs that supposed to be run
"""
for job, job_config in self.jobs_to_do.items():
if job_config.run_always:
if not job_config.has_digest():
continue
pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL)
assert job_config.batches
@ -674,6 +703,50 @@ class CiCache:
bucket=S3_BUILDS_BUCKET, file_path=result_json_path, s3_path=s3_path
)
def filter_out_not_affected_jobs(self):
"""
Filter is to be applied in PRs to remove jobs that are not affected by the change
:return:
"""
remove_from_to_do = []
required_builds = []
for job_name, job_config in self.jobs_to_do.items():
if CI.is_test_job(job_name) and job_name != CI.JobNames.BUILD_CHECK:
if job_config.reference_job_name:
reference_name = job_config.reference_job_name
reference_config = self.jobs_to_do[reference_name]
else:
reference_name = job_name
reference_config = job_config
if self.has_evidence(
job=reference_name,
job_config=reference_config,
):
remove_from_to_do.append(job_name)
else:
required_builds += (
job_config.required_builds if job_config.required_builds else []
)
has_builds_to_do = False
for job_name, job_config in self.jobs_to_do.items():
if CI.is_build_job(job_name):
if job_name not in required_builds:
remove_from_to_do.append(job_name)
else:
has_builds_to_do = True
if not has_builds_to_do:
remove_from_to_do.append(CI.JobNames.BUILD_CHECK)
for job in remove_from_to_do:
print(f"Filter job [{job}] - not affected by the change")
if job in self.jobs_to_do:
del self.jobs_to_do[job]
if job in self.jobs_to_wait:
del self.jobs_to_wait[job]
self.jobs_to_skip.append(job)
def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None:
"""
await pending jobs to be finished

View File

@ -13,6 +13,9 @@ class CI:
each config item in the below dicts should be an instance of JobConfig class or inherited from it
"""
MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI = 5
MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI = 2
# reimport types to CI class so that they visible as CI.* and mypy is happy
# pylint:disable=useless-import-alias,reimported,import-outside-toplevel
from ci_definitions import BuildConfig as BuildConfig
@ -410,7 +413,9 @@ class CI:
release_only=True,
),
JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
reference_job_name=JobNames.INTEGRATION_TEST_TSAN,
),
JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
@ -452,7 +457,10 @@ class CI:
required_builds=[BuildNames.PACKAGE_UBSAN],
),
JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
timeout=3600,
reference_job_name=JobNames.STATELESS_TEST_RELEASE,
),
JobNames.JEPSEN_KEEPER: JobConfig(
required_builds=[BuildNames.BINARY_RELEASE],
@ -637,7 +645,7 @@ class CI:
@classmethod
def is_test_job(cls, job: str) -> bool:
return not cls.is_build_job(job) and job != cls.JobNames.STYLE_CHECK
return not cls.is_build_job(job)
@classmethod
def is_docs_job(cls, job: str) -> bool:

View File

@ -284,8 +284,12 @@ class JobConfig:
# GH Runner type (tag from @Runners)
runner_type: str
# used for config validation in ci unittests
# used in ci unittests for config validation
job_name_keyword: str = ""
# name of another job that (if provided) should be used to check if job was affected by the change or not (in CiCache.has_evidence(job=@reference_job_name) call)
# for example: "Stateless flaky check" can use reference_job_name="Stateless tests (release)". "Stateless flaky check" does not run on master
# and there cannot be an evidence for it, so instead "Stateless tests (release)" job name can be used to check the evidence
reference_job_name: str = ""
# builds required for the job (applicable for test jobs)
required_builds: Optional[List[str]] = None
# build config for the build job (applicable for builds)
@ -327,6 +331,9 @@ class JobConfig:
assert self.required_builds
return self.required_builds[0]
def has_digest(self) -> bool:
return self.digest != DigestConfig()
class CommonJobConfigs:
"""
@ -378,7 +385,7 @@ class CommonJobConfigs:
),
run_command='functional_test_check.py "$CHECK_NAME"',
runner_type=Runners.FUNC_TESTER,
timeout=10800,
timeout=7200,
)
STATEFUL_TEST = JobConfig(
job_name_keyword="stateful",
@ -440,7 +447,12 @@ class CommonJobConfigs:
)
ASTFUZZER_TEST = JobConfig(
job_name_keyword="ast",
digest=DigestConfig(),
digest=DigestConfig(
include_paths=[
"./tests/ci/ast_fuzzer_check.py",
],
docker=["clickhouse/fuzzer"],
),
run_command="ast_fuzzer_check.py",
run_always=True,
runner_type=Runners.FUZZER_UNIT_TESTER,

View File

@ -1,8 +1,9 @@
import os
import re
import subprocess
from contextlib import contextmanager
from pathlib import Path
from typing import Any, Iterator, List, Union
from typing import Any, Iterator, List, Union, Optional
class WithIter(type):
@ -83,3 +84,27 @@ class Shell:
check=False,
)
return result.returncode == 0
class Utils:
@staticmethod
def get_failed_tests_number(description: str) -> Optional[int]:
description = description.lower()
pattern = r"fail:\s*(\d+)\s*(?=,|$)"
match = re.search(pattern, description)
if match:
return int(match.group(1))
return None
@staticmethod
def is_killed_with_oom():
if Shell.check(
"sudo dmesg -T | grep -q -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE'"
):
return True
return False
@staticmethod
def clear_dmesg():
Shell.run("sudo dmesg --clear ||:")

710
tests/ci/create_release.py Executable file
View File

@ -0,0 +1,710 @@
import argparse
import dataclasses
import json
import os
import subprocess
from contextlib import contextmanager
from copy import copy
from pathlib import Path
from typing import Iterator, List
from git_helper import Git, GIT_PREFIX
from ssh import SSHAgent
from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET
from s3_helper import S3Helper
from autoscale_runners_lambda.lambda_shared.pr import Labels
from version_helper import (
FILE_WITH_VERSION_PATH,
GENERATED_CONTRIBUTORS,
get_abs_path,
get_version_from_repo,
update_cmake_version,
update_contributors,
VersionType,
)
from ci_config import CI
CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH)
CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS)
class ShellRunner:
@classmethod
def run(
cls, command, check_retcode=True, print_output=True, async_=False, dry_run=False
):
if dry_run:
print(f"Dry-run: Would run shell command: [{command}]")
return 0, ""
print(f"Running shell command: [{command}]")
if async_:
subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with
return 0, ""
result = subprocess.run(
command + " 2>&1",
shell=True,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
text=True,
check=True,
)
if print_output:
print(result.stdout)
if check_retcode:
assert result.returncode == 0, f"Return code [{result.returncode}]"
return result.returncode, result.stdout
@dataclasses.dataclass
class ReleaseInfo:
version: str
release_tag: str
release_branch: str
commit_sha: str
# lts or stable
codename: str
@staticmethod
def from_file(file_path: str) -> "ReleaseInfo":
with open(file_path, "r", encoding="utf-8") as json_file:
res = json.load(json_file)
return ReleaseInfo(**res)
@staticmethod
def prepare(commit_ref: str, release_type: str, outfile: str) -> None:
Path(outfile).parent.mkdir(parents=True, exist_ok=True)
Path(outfile).unlink(missing_ok=True)
version = None
release_branch = None
release_tag = None
codename = None
assert release_type in ("patch", "new")
if release_type == "new":
# check commit_ref is right and on a right branch
ShellRunner.run(
f"git merge-base --is-ancestor origin/{commit_ref} origin/master"
)
with checkout(commit_ref):
_, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}")
# Git() must be inside "with checkout" contextmanager
git = Git()
version = get_version_from_repo(git=git)
release_branch = "master"
expected_prev_tag = f"v{version.major}.{version.minor}.1.1-new"
version.bump().with_description(VersionType.NEW)
assert (
git.latest_tag == expected_prev_tag
), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]"
release_tag = version.describe
codename = (
VersionType.STABLE
) # dummy value (artifactory won't be updated for new release)
if release_type == "patch":
with checkout(commit_ref):
_, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}")
# Git() must be inside "with checkout" contextmanager
git = Git()
version = get_version_from_repo(git=git)
codename = version.get_stable_release_type()
version.with_description(codename)
release_branch = f"{version.major}.{version.minor}"
release_tag = version.describe
ShellRunner.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags")
# check commit is right and on a right branch
ShellRunner.run(
f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}"
)
if version.patch == 1:
expected_version = copy(version)
expected_version.bump()
expected_tag_prefix = (
f"v{expected_version.major}.{expected_version.minor}-"
)
expected_tag_suffix = "-new"
else:
expected_tag_prefix = (
f"v{version.major}.{version.minor}.{version.patch-1}."
)
expected_tag_suffix = f"-{version.get_stable_release_type()}"
if git.latest_tag.startswith(
expected_tag_prefix
) and git.latest_tag.endswith(expected_tag_suffix):
pass
else:
assert (
False
), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]"
assert (
release_branch
and commit_sha
and release_tag
and version
and codename in ("lts", "stable")
)
res = ReleaseInfo(
release_branch=release_branch,
commit_sha=commit_sha,
release_tag=release_tag,
version=version.string,
codename=codename,
)
with open(outfile, "w", encoding="utf-8") as f:
print(json.dumps(dataclasses.asdict(res), indent=2), file=f)
def push_release_tag(self, dry_run: bool) -> None:
if dry_run:
# remove locally created tag from prev run
ShellRunner.run(
f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag} ||:"
)
# Create release tag
print(
f"Create and push release tag [{self.release_tag}], commit [{self.commit_sha}]"
)
tag_message = f"Release {self.release_tag}"
ShellRunner.run(
f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}"
)
cmd_push_tag = f"{GIT_PREFIX} push origin {self.release_tag}:{self.release_tag}"
ShellRunner.run(cmd_push_tag, dry_run=dry_run)
@staticmethod
def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None:
cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}"
ShellRunner.run(cmd, dry_run=dry_run)
def push_new_release_branch(self, dry_run: bool) -> None:
assert (
self.release_branch == "master"
), "New release branch can be created only for release type [new]"
git = Git()
version = get_version_from_repo(git=git)
new_release_branch = f"{version.major}.{version.minor}"
stable_release_type = version.get_stable_release_type()
version_after_release = copy(version)
version_after_release.bump()
assert (
version_after_release.string == self.version
), f"Unexpected current version in git, must precede [{self.version}] by one step, actual [{version.string}]"
if dry_run:
# remove locally created branch from prev run
ShellRunner.run(
f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch} ||:"
)
print(
f"Create and push new release branch [{new_release_branch}], commit [{self.commit_sha}]"
)
with checkout(self.release_branch):
with checkout_new(new_release_branch):
pr_labels = f"--label {Labels.RELEASE}"
if stable_release_type == VersionType.LTS:
pr_labels += f" --label {Labels.RELEASE_LTS}"
cmd_push_branch = (
f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}"
)
ShellRunner.run(cmd_push_branch, dry_run=dry_run)
print("Create and push backport tags for new release branch")
ReleaseInfo._create_gh_label(
f"v{new_release_branch}-must-backport", "10dbed", dry_run=dry_run
)
ReleaseInfo._create_gh_label(
f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run
)
ShellRunner.run(
f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}'
--head {new_release_branch} {pr_labels}
--body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.'
""",
dry_run=dry_run,
)
def update_version_and_contributors_list(self, dry_run: bool) -> None:
# Bump version, update contributors list, create PR
branch_upd_version_contributors = f"bump_version_{self.version}"
with checkout(self.commit_sha):
git = Git()
version = get_version_from_repo(git=git)
if self.release_branch == "master":
version.bump()
version.with_description(VersionType.TESTING)
else:
version.with_description(version.get_stable_release_type())
assert (
version.string == self.version
), f"BUG: version in release info does not match version in git commit, expected [{self.version}], got [{version.string}]"
with checkout(self.release_branch):
with checkout_new(branch_upd_version_contributors):
update_cmake_version(version)
update_contributors(raise_error=True)
cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'"
cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}"
body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md")
actor = os.getenv("GITHUB_ACTOR", "") or "me"
cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file '{body_file} --label 'do not test' --assignee @{actor}"
ShellRunner.run(cmd_commit_version_upd, dry_run=dry_run)
ShellRunner.run(cmd_push_branch, dry_run=dry_run)
ShellRunner.run(cmd_create_pr, dry_run=dry_run)
if dry_run:
ShellRunner.run(
f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'"
)
ShellRunner.run(
f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'"
)
def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None:
repo = os.getenv("GITHUB_REPOSITORY")
assert repo
cmds = []
cmds.append(
f"gh release create --repo {repo} --title 'Release {self.release_tag}' {self.release_tag}"
)
for file in packages_files:
cmds.append(f"gh release upload {self.release_tag} {file}")
if not dry_run:
for cmd in cmds:
ShellRunner.run(cmd)
else:
print("Dry-run, would run commands:")
print("\n * ".join(cmds))
class RepoTypes:
RPM = "rpm"
DEBIAN = "deb"
TGZ = "tgz"
class PackageDownloader:
PACKAGES = (
"clickhouse-client",
"clickhouse-common-static",
"clickhouse-common-static-dbg",
"clickhouse-keeper",
"clickhouse-keeper-dbg",
"clickhouse-server",
)
EXTRA_PACKAGES = (
"clickhouse-library-bridge",
"clickhouse-odbc-bridge",
)
PACKAGE_TYPES = (CI.BuildNames.PACKAGE_RELEASE, CI.BuildNames.PACKAGE_AARCH64)
MACOS_PACKAGE_TO_BIN_SUFFIX = {
CI.BuildNames.BINARY_DARWIN: "macos",
CI.BuildNames.BINARY_DARWIN_AARCH64: "macos-aarch64",
}
LOCAL_DIR = "/tmp/packages"
@classmethod
def _get_arch_suffix(cls, package_arch, repo_type):
if package_arch == CI.BuildNames.PACKAGE_RELEASE:
return (
"amd64" if repo_type in (RepoTypes.DEBIAN, RepoTypes.TGZ) else "x86_64"
)
elif package_arch == CI.BuildNames.PACKAGE_AARCH64:
return (
"arm64" if repo_type in (RepoTypes.DEBIAN, RepoTypes.TGZ) else "aarch64"
)
else:
assert False, "BUG"
def __init__(self, release, commit_sha, version):
assert version.startswith(release), "Invalid release branch or version"
major, minor = map(int, release.split("."))
self.package_names = list(self.PACKAGES)
if major > 24 or (major == 24 and minor > 3):
self.package_names += list(self.EXTRA_PACKAGES)
self.release = release
self.commit_sha = commit_sha
self.version = version
self.s3 = S3Helper()
self.deb_package_files = []
self.rpm_package_files = []
self.tgz_package_files = []
# just binaries for macos
self.macos_package_files = ["clickhouse-macos", "clickhouse-macos-aarch64"]
self.file_to_type = {}
ShellRunner.run(f"mkdir -p {self.LOCAL_DIR}")
for package_type in self.PACKAGE_TYPES:
for package in self.package_names:
deb_package_file_name = f"{package}_{self.version}_{self._get_arch_suffix(package_type, RepoTypes.DEBIAN)}.deb"
self.deb_package_files.append(deb_package_file_name)
self.file_to_type[deb_package_file_name] = package_type
rpm_package_file_name = f"{package}-{self.version}.{self._get_arch_suffix(package_type, RepoTypes.RPM)}.rpm"
self.rpm_package_files.append(rpm_package_file_name)
self.file_to_type[rpm_package_file_name] = package_type
tgz_package_file_name = f"{package}-{self.version}-{self._get_arch_suffix(package_type, RepoTypes.TGZ)}.tgz"
self.tgz_package_files.append(tgz_package_file_name)
self.file_to_type[tgz_package_file_name] = package_type
tgz_package_file_name += ".sha512"
self.tgz_package_files.append(tgz_package_file_name)
self.file_to_type[tgz_package_file_name] = package_type
def get_deb_packages_files(self):
return self.deb_package_files
def get_rpm_packages_files(self):
return self.rpm_package_files
def get_tgz_packages_files(self):
return self.tgz_package_files
def get_macos_packages_files(self):
return self.macos_package_files
def get_packages_names(self):
return self.package_names
def get_all_packages_files(self):
assert self.local_tgz_packages_ready()
assert self.local_deb_packages_ready()
assert self.local_rpm_packages_ready()
assert self.local_macos_packages_ready()
res = []
for package_file in (
self.deb_package_files
+ self.rpm_package_files
+ self.tgz_package_files
+ self.macos_package_files
):
res.append(self.LOCAL_DIR + "/" + package_file)
return res
def run(self):
ShellRunner.run(f"rm -rf {self.LOCAL_DIR}/*")
for package_file in (
self.deb_package_files + self.rpm_package_files + self.tgz_package_files
):
print(f"Downloading: [{package_file}]")
s3_path = "/".join(
[
self.release,
self.commit_sha,
self.file_to_type[package_file],
package_file,
]
)
self.s3.download_file(
bucket=S3_BUILDS_BUCKET,
s3_path=s3_path,
local_file_path="/".join([self.LOCAL_DIR, package_file]),
)
for macos_package, bin_suffix in self.MACOS_PACKAGE_TO_BIN_SUFFIX.items():
binary_name = "clickhouse"
destination_binary_name = f"{binary_name}-{bin_suffix}"
assert destination_binary_name in self.macos_package_files
print(
f"Downloading: [{macos_package}] binary to [{destination_binary_name}]"
)
s3_path = "/".join(
[
self.release,
self.commit_sha,
macos_package,
binary_name,
]
)
self.s3.download_file(
bucket=S3_BUILDS_BUCKET,
s3_path=s3_path,
local_file_path="/".join([self.LOCAL_DIR, destination_binary_name]),
)
def local_deb_packages_ready(self) -> bool:
assert self.deb_package_files
for package_file in self.deb_package_files:
print(f"Check package is downloaded [{package_file}]")
if not Path(self.LOCAL_DIR + "/" + package_file).is_file():
return False
return True
def local_rpm_packages_ready(self) -> bool:
assert self.rpm_package_files
for package_file in self.rpm_package_files:
print(f"Check package is downloaded [{package_file}]")
if not Path(self.LOCAL_DIR + "/" + package_file).is_file():
return False
return True
def local_tgz_packages_ready(self) -> bool:
assert self.tgz_package_files
for package_file in self.tgz_package_files:
print(f"Check package is downloaded [{package_file}]")
if not Path(self.LOCAL_DIR + "/" + package_file).is_file():
return False
return True
def local_macos_packages_ready(self) -> bool:
assert self.macos_package_files
for package_file in self.macos_package_files:
print(f"Check package is downloaded [{package_file}]")
if not Path(self.LOCAL_DIR + "/" + package_file).is_file():
return False
return True
def parse_args() -> argparse.Namespace:
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Creates release",
)
parser.add_argument(
"--prepare-release-info",
action="store_true",
help="Initial step to prepare info like release branch, release tag, etc.",
)
parser.add_argument(
"--push-release-tag",
action="store_true",
help="Creates and pushes git tag",
)
parser.add_argument(
"--push-new-release-branch",
action="store_true",
help="Creates and pushes new release branch and corresponding service gh tags for backports",
)
parser.add_argument(
"--create-bump-version-pr",
action="store_true",
help="Updates version, contributors' list and creates PR",
)
parser.add_argument(
"--download-packages",
action="store_true",
help="Downloads all required packages from s3",
)
parser.add_argument(
"--create-gh-release",
action="store_true",
help="Create GH Release object and attach all packages",
)
parser.add_argument(
"--ref",
type=str,
help="the commit hash or branch",
)
parser.add_argument(
"--release-type",
choices=("new", "patch"),
# dest="release_type",
help="a release type to bump the major.minor.patch version part, "
"new branch is created only for the value 'new'",
)
parser.add_argument(
"--dry-run",
action="store_true",
help="do not make any actual changes in the repo, just show what will be done",
)
parser.add_argument(
"--outfile",
default="",
type=str,
help="output file to write json result to, if not set - stdout",
)
parser.add_argument(
"--infile",
default="",
type=str,
help="input file with release info",
)
return parser.parse_args()
@contextmanager
def checkout(ref: str) -> Iterator[None]:
_, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD")
rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}"
assert orig_ref
if ref not in (orig_ref,):
ShellRunner.run(f"{GIT_PREFIX} checkout {ref}")
try:
yield
except (Exception, KeyboardInterrupt) as e:
print(f"ERROR: Exception [{e}]")
ShellRunner.run(rollback_cmd)
raise
ShellRunner.run(rollback_cmd)
@contextmanager
def checkout_new(ref: str) -> Iterator[None]:
_, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD")
rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}"
assert orig_ref
ShellRunner.run(f"{GIT_PREFIX} checkout -b {ref}")
try:
yield
except (Exception, KeyboardInterrupt) as e:
print(f"ERROR: Exception [{e}]")
ShellRunner.run(rollback_cmd)
raise
ShellRunner.run(rollback_cmd)
if __name__ == "__main__":
args = parse_args()
assert args.dry_run
# prepare ssh for git if needed
_ssh_agent = None
_key_pub = None
if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""):
_key = os.getenv("ROBOT_CLICKHOUSE_SSH_KEY")
_ssh_agent = SSHAgent()
_key_pub = _ssh_agent.add(_key)
_ssh_agent.print_keys()
if args.prepare_release_info:
assert (
args.ref and args.release_type and args.outfile
), "--ref, --release-type and --outfile must be provided with --prepare-release-info"
ReleaseInfo.prepare(
commit_ref=args.ref, release_type=args.release_type, outfile=args.outfile
)
if args.push_release_tag:
assert args.infile, "--infile <release info file path> must be provided"
release_info = ReleaseInfo.from_file(args.infile)
release_info.push_release_tag(dry_run=args.dry_run)
if args.push_new_release_branch:
assert args.infile, "--infile <release info file path> must be provided"
release_info = ReleaseInfo.from_file(args.infile)
release_info.push_new_release_branch(dry_run=args.dry_run)
if args.create_bump_version_pr:
# TODO: store link to PR in release info
assert args.infile, "--infile <release info file path> must be provided"
release_info = ReleaseInfo.from_file(args.infile)
release_info.update_version_and_contributors_list(dry_run=args.dry_run)
if args.download_packages:
assert args.infile, "--infile <release info file path> must be provided"
release_info = ReleaseInfo.from_file(args.infile)
p = PackageDownloader(
release=release_info.release_branch,
commit_sha=release_info.commit_sha,
version=release_info.version,
)
p.run()
if args.create_gh_release:
assert args.infile, "--infile <release info file path> must be provided"
release_info = ReleaseInfo.from_file(args.infile)
p = PackageDownloader(
release=release_info.release_branch,
commit_sha=release_info.commit_sha,
version=release_info.version,
)
release_info.create_gh_release(p.get_all_packages_files(), args.dry_run)
# tear down ssh
if _ssh_agent and _key_pub:
_ssh_agent.remove(_key_pub)
"""
Prepare release machine:
### INSTALL PACKAGES
sudo apt update
sudo apt install --yes --no-install-recommends python3-dev python3-pip gh unzip
sudo apt install --yes python3-boto3
sudo apt install --yes python3-github
sudo apt install --yes python3-unidiff
sudo apt install --yes s3fs
### INSTALL AWS CLI
cd /tmp
curl "https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m).zip" -o "awscliv2.zip"
unzip awscliv2.zip
sudo ./aws/install
rm -rf aws*
cd -
### INSTALL GH ACTIONS RUNNER:
# Create a folder
RUNNER_VERSION=2.317.0
cd ~
mkdir actions-runner && cd actions-runner
# Download the latest runner package
runner_arch() {
case $(uname -m) in
x86_64 )
echo x64;;
aarch64 )
echo arm64;;
esac
}
curl -O -L https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz
# Extract the installer
tar xzf ./actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz
rm ./actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz
### Install reprepro:
cd ~
sudo apt install dpkg-dev libgpgme-dev libdb-dev libbz2-dev liblzma-dev libarchive-dev shunit2 db-util debhelper
git clone https://salsa.debian.org/debian/reprepro.git
cd reprepro
dpkg-buildpackage -b --no-sign && sudo dpkg -i ../reprepro_$(dpkg-parsechangelog --show-field Version)_$(dpkg-architecture -q DEB_HOST_ARCH).deb
### Install createrepo-c:
sudo apt install createrepo-c
createrepo_c --version
#Version: 0.17.3 (Features: DeltaRPM LegacyWeakdeps )
### Import gpg sign key
gpg --import key.pgp
gpg --list-secret-keys
### Install docker
sudo su; cd ~
deb_arch() {
case $(uname -m) in
x86_64 )
echo amd64;;
aarch64 )
echo arm64;;
esac
}
curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg
echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | sudo tee /etc/apt/sources.list.d/docker.list > /dev/null
sudo apt-get update
sudo apt-get install --yes --no-install-recommends docker-ce docker-buildx-plugin docker-ce-cli containerd.io
sudo usermod -aG docker ubuntu
# enable ipv6 in containers (fixed-cidr-v6 is some random network mask)
cat <<EOT > /etc/docker/daemon.json
{
"ipv6": true,
"fixed-cidr-v6": "2001:db8:1::/64",
"log-driver": "json-file",
"log-opts": {
"max-file": "5",
"max-size": "1000m"
},
"insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"]
}
EOT
# if docker build does not work:
sudo systemctl restart docker
docker buildx rm mybuilder
docker buildx create --name mybuilder --driver docker-container --use
docker buildx inspect mybuilder --bootstrap
### Install tailscale
### Configure GH runner
"""

View File

@ -11,7 +11,6 @@ from os import path as p
from pathlib import Path
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 (
@ -22,7 +21,7 @@ from env_helper import (
TEMP_PATH,
)
from git_helper import Git
from pr_info import PRInfo
from pr_info import PRInfo, EventType
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -63,6 +62,12 @@ def parse_args() -> argparse.Namespace:
help="a version to build, automaticaly got from version_helper, accepts either "
"tag ('refs/tags/' is removed automatically) or a normal 22.2.2.2 format",
)
parser.add_argument(
"--sha",
type=str,
default="",
help="sha of the commit to use packages from",
)
parser.add_argument(
"--release-type",
type=str,
@ -122,7 +127,7 @@ def parse_args() -> argparse.Namespace:
def retry_popen(cmd: str, log_file: Path) -> int:
max_retries = 5
max_retries = 2
for retry in range(max_retries):
# From time to time docker build may failed. Curl issues, or even push
# It will sleep progressively 5, 15, 30 and 50 seconds between retries
@ -370,13 +375,22 @@ def main():
tags = gen_tags(args.version, args.release_type)
repo_urls = {}
direct_urls: Dict[str, List[str]] = {}
release_or_pr, _ = get_release_or_pr(pr_info, args.version)
if pr_info.event_type == EventType.PULL_REQUEST:
release_or_pr = str(pr_info.number)
sha = pr_info.sha
elif pr_info.event_type == EventType.PUSH and pr_info.is_master:
release_or_pr = str(0)
sha = pr_info.sha
else:
release_or_pr = f"{args.version.major}.{args.version.minor}"
sha = args.sha
assert sha
for arch, build_name in zip(ARCH, ("package_release", "package_aarch64")):
if not args.bucket_prefix:
repo_urls[arch] = (
f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/"
f"{release_or_pr}/{pr_info.sha}/{build_name}"
f"{release_or_pr}/{sha}/{build_name}"
)
else:
repo_urls[arch] = f"{args.bucket_prefix}/{build_name}"

View File

@ -108,6 +108,7 @@ def get_run_command(
"--privileged "
f"{ci_logs_args}"
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
f"--volume={repo_path}/utils/grpc-client:/usr/share/clickhouse-utils/grpc-client "
f"{volume_with_broken_test}"
f"--volume={result_path}:/test_output "
f"--volume={server_log_path}:/var/log/clickhouse-server "

View File

@ -1,4 +1,5 @@
"""Module to get the token for GitHub"""
from dataclasses import dataclass
import json
import time

View File

@ -26,6 +26,8 @@ from pr_info import PRInfo
from report import SUCCESS, FAILURE
from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY
from synchronizer_utils import SYNC_BRANCH_PREFIX
from ci_config import CI
from ci_utils import Utils
# The team name for accepted approvals
TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core")
@ -251,23 +253,77 @@ def main():
# set mergeable check status and exit
commit = get_commit(gh, args.pr_info.sha)
statuses = get_commit_filtered_statuses(commit)
state = trigger_mergeable_check(
commit,
statuses,
workflow_failed=(args.wf_status != "success"),
)
# Process upstream StatusNames.SYNC
pr_info = PRInfo()
if (
pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/")
and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY
):
print("Updating upstream statuses")
update_upstream_sync_status(pr_info, state)
max_failed_tests_per_job = 0
job_name_with_max_failures = None
total_failed_tests = 0
failed_to_get_info = False
has_failed_statuses = False
for status in statuses:
if not CI.is_required(status.context):
continue
if status.state == FAILURE:
has_failed_statuses = True
failed_cnt = Utils.get_failed_tests_number(status.description)
if failed_cnt is None:
failed_to_get_info = True
else:
if failed_cnt > max_failed_tests_per_job:
job_name_with_max_failures = status.context
max_failed_tests_per_job = failed_cnt
total_failed_tests += failed_cnt
elif status.state != SUCCESS and status.context not in (
CI.StatusNames.SYNC,
CI.StatusNames.PR_CHECK,
):
# do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK)
has_failed_statuses = True
print(
f"Unexpected status for [{status.context}]: [{status.state}] - block further testing"
)
failed_to_get_info = True
if args.wf_status != "success":
# exit with 1 to rerun on workflow failed job restart
can_continue = True
if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI:
print(
f"Required check has [{total_failed_tests}] failed - block further testing"
)
can_continue = False
if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI:
print(
f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing"
)
can_continue = False
if failed_to_get_info:
print("Unexpected commit status state - block further testing")
can_continue = False
if args.wf_status != SUCCESS and not has_failed_statuses:
# workflow failed but reason is unknown as no failed statuses present
can_continue = False
print(
"WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation."
)
if args.wf_status == SUCCESS or has_failed_statuses:
# do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted
state = trigger_mergeable_check(
commit,
statuses,
)
# Process upstream StatusNames.SYNC
pr_info = PRInfo()
if (
pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/")
and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY
):
print("Updating upstream statuses")
update_upstream_sync_status(pr_info, state)
else:
print(
"Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status"
)
if not can_continue:
sys.exit(1)
sys.exit(0)

View File

@ -23,7 +23,7 @@ from typing import (
from build_download_helper import get_gh_api
from ci_config import CI
from ci_utils import normalize_string
from env_helper import REPORT_PATH, TEMP_PATH
from env_helper import REPORT_PATH, GITHUB_WORKSPACE
logger = logging.getLogger(__name__)
@ -244,7 +244,8 @@ HTML_TEST_PART = """
"""
BASE_HEADERS = ["Test name", "Test status"]
JOB_REPORT_FILE = Path(TEMP_PATH) / "job_report.json"
# should not be in TEMP directory or any directory that may be cleaned during the job execution
JOB_REPORT_FILE = Path(GITHUB_WORKSPACE) / "job_report.json"
@dataclass
@ -296,6 +297,33 @@ class JobReport:
build_dir_for_upload: Union[Path, str] = ""
# if False no GH commit status will be created by CI
need_commit_status: bool = True
# indicates that this is not real job report but report for the job that was skipped by rerun check
job_skipped: bool = False
# indicates that report generated by CI script in order to check later if job was killed before real report is generated
pre_report: bool = False
exit_code: int = -1
@staticmethod
def create_pre_report() -> "JobReport":
return JobReport(
status=ERROR,
description="",
test_results=[],
start_time=datetime.datetime.utcnow().strftime("%Y-%m-%d %H:%M:%S"),
duration=0.0,
additional_files=[],
pre_report=True,
)
def update_duration(self):
if not self.start_time:
self.duration = 0.0
else:
start_time = datetime.datetime.strptime(
self.start_time, "%Y-%m-%d %H:%M:%S"
)
current_time = datetime.datetime.utcnow()
self.duration = (current_time - start_time).total_seconds()
def __post_init__(self):
assert self.status in (SUCCESS, ERROR, FAILURE, PENDING)

View File

@ -37,9 +37,9 @@ class SSHAgent:
ssh_options = (
"," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else ""
)
os.environ[
"SSH_OPTIONS"
] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no"
os.environ["SSH_OPTIONS"] = (
f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no"
)
def add(self, key):
key_pub = self._key_pub(key)

View File

@ -16,7 +16,15 @@ from docker_images_helper import get_docker_image, pull_image
from env_helper import IS_CI, REPO_COPY, TEMP_PATH, GITHUB_EVENT_PATH
from git_helper import GIT_PREFIX, git_runner
from pr_info import PRInfo
from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results
from report import (
ERROR,
FAILURE,
SUCCESS,
JobReport,
TestResults,
read_test_results,
FAIL,
)
from ssh import SSHKey
from stopwatch import Stopwatch
@ -192,15 +200,6 @@ def main():
future = executor.submit(subprocess.run, cmd_shell, shell=True)
_ = future.result()
autofix_description = ""
if args.push:
try:
commit_push_staged(pr_info)
except subprocess.SubprocessError:
# do not fail the whole script if the autofix didn't work out
logging.error("Unable to push the autofix. Continue.")
autofix_description = "Failed to push autofix to the PR. "
subprocess.check_call(
f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} "
f"--out-results-file {temp_path}/test_results.tsv --out-status-file {temp_path}/check_status.tsv || "
@ -210,6 +209,21 @@ def main():
state, description, test_results, additional_files = process_result(temp_path)
autofix_description = ""
fail_cnt = 0
for result in test_results:
if result.status in (FAILURE, FAIL):
# do not autofix if not only black failed
fail_cnt += 1
if args.push and fail_cnt == 1:
try:
commit_push_staged(pr_info)
except subprocess.SubprocessError:
# do not fail the whole script if the autofix didn't work out
logging.error("Unable to push the autofix. Continue.")
autofix_description = "Failed to push autofix to the PR. "
JobReport(
description=f"{autofix_description}{description}",
test_results=test_results,

View File

@ -1,6 +1,8 @@
#!/usr/bin/env python3
import unittest
import random
from ci_config import CI
import ci as CIPY
from ci_settings import CiSettings
@ -57,6 +59,18 @@ class TestCIConfig(unittest.TestCase):
f"Job [{job}] apparently uses wrong common config with job keyword [{CI.JOB_CONFIGS[job].job_name_keyword}]",
)
def test_job_config_has_proper_values(self):
for job in CI.JobNames:
if CI.JOB_CONFIGS[job].reference_job_name:
reference_job_config = CI.JOB_CONFIGS[
CI.JOB_CONFIGS[job].reference_job_name
]
# reference job must run in all workflows and has digest
self.assertTrue(reference_job_config.pr_only == False)
self.assertTrue(reference_job_config.release_only == False)
self.assertTrue(reference_job_config.run_always == False)
self.assertTrue(reference_job_config.digest != CI.DigestConfig())
def test_required_checks(self):
for job in CI.REQUIRED_CHECKS:
if job in (CI.StatusNames.PR_CHECK, CI.StatusNames.SYNC):
@ -417,7 +431,7 @@ class TestCIConfig(unittest.TestCase):
assert not ci_cache.jobs_to_skip
assert not ci_cache.jobs_to_wait
# pretend there are pending jobs that we neet to wait
# pretend there are pending jobs that we need to wait
ci_cache.jobs_to_wait = dict(ci_cache.jobs_to_do)
for job, config in ci_cache.jobs_to_wait.items():
assert not config.pending_batches
@ -489,3 +503,76 @@ class TestCIConfig(unittest.TestCase):
self.assertCountEqual(
list(ci_cache.jobs_to_do) + ci_cache.jobs_to_skip, all_jobs_in_wf
)
def test_ci_py_filters_not_affected_jobs_in_prs(self):
"""
checks ci.py filters not affected jobs in PRs
"""
settings = CiSettings()
settings.no_ci_cache = True
pr_info = PRInfo(github_event=_TEST_EVENT_JSON)
pr_info.event_type = EventType.PULL_REQUEST
pr_info.number = 123
assert pr_info.is_pr
ci_cache = CIPY._configure_jobs(
S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True
)
self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list")
assert not ci_cache.jobs_to_wait
assert not ci_cache.jobs_to_skip
MOCK_AFFECTED_JOBS = [
CI.JobNames.STATELESS_TEST_S3_DEBUG,
CI.JobNames.STRESS_TEST_TSAN,
]
MOCK_REQUIRED_BUILDS = []
# pretend there are pending jobs that we need to wait
for job, job_config in ci_cache.jobs_to_do.items():
if job in MOCK_AFFECTED_JOBS:
MOCK_REQUIRED_BUILDS += job_config.required_builds
elif job not in MOCK_AFFECTED_JOBS:
ci_cache.jobs_to_wait[job] = job_config
for job, job_config in ci_cache.jobs_to_do.items():
if job_config.reference_job_name:
# jobs with reference_job_name in config are not supposed to have records in the cache - continue
continue
if job in MOCK_AFFECTED_JOBS:
continue
for batch in range(job_config.num_batches):
# add any record into cache
record = CiCache.Record(
record_type=random.choice(
[
CiCache.RecordType.FAILED,
CiCache.RecordType.PENDING,
CiCache.RecordType.SUCCESSFUL,
]
),
job_name=job,
job_digest=ci_cache.job_digests[job],
batch=batch,
num_batches=job_config.num_batches,
release_branch=True,
)
for record_t_, records_ in ci_cache.records.items():
if record_t_.value == CiCache.RecordType.FAILED.value:
records_[record.to_str_key()] = record
ci_cache.filter_out_not_affected_jobs()
expected_to_do = (
[
CI.JobNames.BUILD_CHECK,
]
+ MOCK_AFFECTED_JOBS
+ MOCK_REQUIRED_BUILDS
)
self.assertCountEqual(
list(ci_cache.jobs_to_wait),
[
CI.JobNames.BUILD_CHECK,
]
+ MOCK_REQUIRED_BUILDS,
)
self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do)

View File

@ -172,14 +172,10 @@ class TestCIOptions(unittest.TestCase):
job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER)
for job in _TEST_JOB_LIST
}
jobs_configs[
"fuzzers"
].run_by_label = (
jobs_configs["fuzzers"].run_by_label = (
"TEST_LABEL" # check "fuzzers" appears in the result due to the label
)
jobs_configs[
"Integration tests (asan)"
].release_only = (
jobs_configs["Integration tests (asan)"].release_only = (
True # still must be included as it's set with include keywords
)
filtered_jobs = list(
@ -311,9 +307,9 @@ class TestCIOptions(unittest.TestCase):
job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER)
for job in _TEST_JOB_LIST
}
jobs_configs[
"fuzzers"
].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result
jobs_configs["fuzzers"].run_by_label = (
"TEST_LABEL" # check "fuzzers" does not appears in the result
)
jobs_configs["Integration tests (asan)"].release_only = True
filtered_jobs = list(
ci_options.apply(

View File

@ -72,6 +72,19 @@ class ClickHouseVersion:
return self.patch_update()
raise KeyError(f"wrong part {part} is used")
def bump(self) -> "ClickHouseVersion":
if self.minor < 12:
self._minor += 1
self._revision += 1
self._patch = 1
self._tweak = 1
else:
self._major += 1
self._revision += 1
self._patch = 1
self._tweak = 1
return self
def major_update(self) -> "ClickHouseVersion":
if self._git is not None:
self._git.update()
@ -148,6 +161,11 @@ class ClickHouseVersion:
"""our X.3 and X.8 are LTS"""
return self.minor % 5 == 3
def get_stable_release_type(self) -> str:
if self.is_lts:
return VersionType.LTS
return VersionType.STABLE
def as_dict(self) -> VERSIONS:
return {
"revision": self.revision,
@ -168,6 +186,7 @@ class ClickHouseVersion:
raise ValueError(f"version type {version_type} not in {VersionType.VALID}")
self._description = version_type
self._describe = f"v{self.string}-{version_type}"
return self
def copy(self) -> "ClickHouseVersion":
copy = ClickHouseVersion(

View File

@ -711,9 +711,9 @@ def get_localzone():
class SettingsRandomizer:
settings = {
"max_insert_threads": lambda: 32
if random.random() < 0.03
else random.randint(1, 3),
"max_insert_threads": lambda: (
12 if random.random() < 0.03 else random.randint(1, 3)
),
"group_by_two_level_threshold": threshold_generator(0.2, 0.2, 1, 1000000),
"group_by_two_level_threshold_bytes": threshold_generator(
0.2, 0.2, 1, 50000000
@ -729,7 +729,7 @@ class SettingsRandomizer:
"prefer_localhost_replica": lambda: random.randint(0, 1),
"max_block_size": lambda: random.randint(8000, 100000),
"max_joined_block_size_rows": lambda: random.randint(8000, 100000),
"max_threads": lambda: 64 if random.random() < 0.03 else random.randint(1, 3),
"max_threads": lambda: 32 if random.random() < 0.03 else random.randint(1, 3),
"optimize_append_index": lambda: random.randint(0, 1),
"optimize_if_chain_to_multiif": lambda: random.randint(0, 1),
"optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1),
@ -1759,7 +1759,7 @@ class TestCase:
return TestResult(
self.name,
TestStatus.FAIL,
FailureReason.INTERNAL_QUERY_FAIL,
FailureReason.TIMEOUT,
total_time,
self.add_info_about_settings(
self.get_description_from_exception_info(sys.exc_info())
@ -2198,11 +2198,26 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool
sys.stdout.flush()
while True:
test_result = test_case.run(
args, test_suite, client_options, server_logs_level
)
test_result = test_case.process_result(test_result, MESSAGES)
if not test_result.need_retry:
# This is the upper level timeout
# It helps with completely frozen processes, like in case of gdb errors
def timeout_handler(signum, frame):
raise TimeoutError("Test execution timed out")
signal.signal(signal.SIGALRM, timeout_handler)
signal.alarm(int(args.timeout * 1.1))
test_result = None
try:
test_result = test_case.run(
args, test_suite, client_options, server_logs_level
)
test_result = test_case.process_result(test_result, MESSAGES)
break
except TimeoutError:
break
finally:
signal.alarm(0)
if not test_result or not test_result.need_retry:
break
restarted_tests.append(test_result)
@ -2461,6 +2476,10 @@ def override_envs(*args_, **kwargs):
run_tests_array(*args_, **kwargs)
def run_tests_process(*args, **kwargs):
return run_tests_array(*args, **kwargs)
def do_run_tests(jobs, test_suite: TestSuite):
if jobs > 1 and len(test_suite.parallel_tests) > 0:
print(
@ -2484,39 +2503,70 @@ def do_run_tests(jobs, test_suite: TestSuite):
# of failures will be nearly the same for all tests from the group.
random.shuffle(test_suite.parallel_tests)
batch_size = max(1, len(test_suite.parallel_tests) // jobs)
batch_size = max(1, (len(test_suite.parallel_tests) // jobs) + 1)
parallel_tests_array = []
for job in range(jobs):
range_ = job * batch_size, job * batch_size + batch_size
batch = test_suite.parallel_tests[range_[0] : range_[1]]
parallel_tests_array.append((batch, batch_size, test_suite, True))
try:
with multiprocessing.Pool(processes=jobs + 1) as pool:
future = pool.map_async(run_tests_array, parallel_tests_array)
processes = []
if args.run_sequential_tests_in_parallel:
# Run parallel tests and sequential tests at the same time
# Sequential tests will use different ClickHouse instance
# In this process we can safely override values in `args` and `os.environ`
future_seq = pool.map_async(
override_envs,
[
(
test_suite.sequential_tests,
len(test_suite.sequential_tests),
test_suite,
False,
)
],
)
future_seq.wait()
for test_batch in parallel_tests_array:
process = multiprocessing.Process(
target=run_tests_process, args=(test_batch,)
)
processes.append(process)
process.start()
future.wait()
finally:
pool.terminate()
pool.close()
pool.join()
if args.run_sequential_tests_in_parallel:
# Run parallel tests and sequential tests at the same time
# Sequential tests will use different ClickHouse instance
# In this process we can safely override values in `args` and `os.environ`
process = multiprocessing.Process(
target=override_envs,
args=(
(
test_suite.sequential_tests,
len(test_suite.sequential_tests),
test_suite,
False,
),
),
)
processes.append(process)
process.start()
while processes:
sys.stdout.flush()
# Periodically check the server for hangs
# and stop all processes in this case
try:
clickhouse_execute(
args,
query="SELECT 1 /*hang up check*/",
max_http_retries=5,
timeout=20,
)
except Exception:
print("Hang up check failed")
server_died.set()
if server_died.is_set():
print("Server died, terminating all processes...")
kill_gdb_if_any()
# Wait for test results
sleep(args.timeout)
for p in processes:
if p.is_alive():
p.terminate()
break
for p in processes[:]:
if not p.is_alive():
processes.remove(p)
sleep(5)
if not args.run_sequential_tests_in_parallel:
run_tests_array(
@ -3381,6 +3431,14 @@ def parse_args():
return parser.parse_args()
class Terminated(KeyboardInterrupt):
pass
def signal_handler(sig, frame):
raise Terminated(f"Terminated with {sig} signal")
if __name__ == "__main__":
stop_time = None
exit_code = multiprocessing.Value("i", 0)
@ -3392,6 +3450,9 @@ if __name__ == "__main__":
# infinite tests processes left
# (new process group is required to avoid killing some parent processes)
os.setpgid(0, 0)
signal.signal(signal.SIGTERM, signal_handler)
signal.signal(signal.SIGINT, signal_handler)
signal.signal(signal.SIGHUP, signal_handler)
try:
args = parse_args()

View File

@ -0,0 +1,3 @@
<clickhouse>
<grpc_port>9100</grpc_port>
</clickhouse>

View File

@ -27,6 +27,7 @@ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/graphite_alternative.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/grpc_protocol.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/merge_tree_settings.xml $DEST_SERVER_PATH/config.d/

View File

@ -1454,9 +1454,9 @@ class ClickHouseCluster:
def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_azurite = True
env_variables["AZURITE_PORT"] = str(self.azurite_port)
env_variables[
"AZURITE_STORAGE_ACCOUNT_URL"
] = f"http://azurite1:{env_variables['AZURITE_PORT']}/devstoreaccount1"
env_variables["AZURITE_STORAGE_ACCOUNT_URL"] = (
f"http://azurite1:{env_variables['AZURITE_PORT']}/devstoreaccount1"
)
env_variables["AZURITE_CONNECTION_STRING"] = (
f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;"
f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
@ -1653,9 +1653,9 @@ class ClickHouseCluster:
# Code coverage files will be placed in database directory
# (affect only WITH_COVERAGE=1 build)
env_variables[
"LLVM_PROFILE_FILE"
] = "/var/lib/clickhouse/server_%h_%p_%m.profraw"
env_variables["LLVM_PROFILE_FILE"] = (
"/var/lib/clickhouse/server_%h_%p_%m.profraw"
)
clickhouse_start_command = CLICKHOUSE_START_COMMAND
if clickhouse_log_file:
@ -1668,9 +1668,9 @@ class ClickHouseCluster:
cluster=self,
base_path=self.base_dir,
name=name,
base_config_dir=base_config_dir
if base_config_dir
else self.base_config_dir,
base_config_dir=(
base_config_dir if base_config_dir else self.base_config_dir
),
custom_main_configs=main_configs or [],
custom_user_configs=user_configs or [],
custom_dictionaries=dictionaries or [],

View File

@ -19,9 +19,9 @@ def cluster():
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node",
main_configs=["configs/storage_arm.xml"]
if is_arm()
else ["configs/storage_amd.xml"],
main_configs=(
["configs/storage_arm.xml"] if is_arm() else ["configs/storage_amd.xml"]
),
with_minio=True,
with_hdfs=not is_arm(),
)

View File

@ -5,6 +5,7 @@ in this test we write into per-node tables and read from the distributed table.
The default database in the distributed table definition is left empty on purpose to test
default database deduction.
"""
import pytest
from helpers.client import QueryRuntimeException

View File

@ -1,4 +0,0 @@
<clickhouse>
<disable_internal_dns_cache>1</disable_internal_dns_cache>
<max_concurrent_queries>250</max_concurrent_queries>
</clickhouse>

View File

@ -1,11 +0,0 @@
<clickhouse>
<users>
<test_dns>
<password/>
<networks>
<host_regexp>test1\.example\.com$</host_regexp>
</networks>
<profile>default</profile>
</test_dns>
</users>
</clickhouse>

View File

@ -1,5 +0,0 @@
<clickhouse>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<listen_try>1</listen_try>
</clickhouse>

View File

@ -1,8 +0,0 @@
. {
hosts /example.com {
reload "20ms"
fallthrough
}
forward . 127.0.0.11
log
}

View File

@ -1 +0,0 @@
filled in runtime, but needs to exist in order to be volume mapped in docker

View File

@ -1,62 +0,0 @@
import pycurl
import threading
from io import BytesIO
import sys
client_ip = sys.argv[1]
server_ip = sys.argv[2]
mutex = threading.Lock()
success_counter = 0
number_of_threads = 100
number_of_iterations = 50
def perform_request():
buffer = BytesIO()
crl = pycurl.Curl()
crl.setopt(pycurl.INTERFACE, client_ip)
crl.setopt(crl.WRITEDATA, buffer)
crl.setopt(crl.URL, f"http://{server_ip}:8123/?query=select+1&user=test_dns")
crl.perform()
# End curl session
crl.close()
str_response = buffer.getvalue().decode("iso-8859-1")
expected_response = "1\n"
mutex.acquire()
global success_counter
if str_response == expected_response:
success_counter += 1
mutex.release()
def perform_multiple_requests(n):
for request_number in range(n):
perform_request()
threads = []
for i in range(number_of_threads):
thread = threading.Thread(
target=perform_multiple_requests, args=(number_of_iterations,)
)
thread.start()
threads.append(thread)
for thread in threads:
thread.join()
if success_counter == number_of_threads * number_of_iterations:
exit(0)
exit(1)

View File

@ -1,88 +0,0 @@
import pytest
import socket
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
from time import sleep
import os
DOCKER_COMPOSE_PATH = get_docker_compose_path()
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
ch_server = cluster.add_instance(
"clickhouse-server",
with_coredns=True,
main_configs=["configs/config.xml", "configs/listen_host.xml"],
user_configs=["configs/host_regexp.xml"],
)
client = cluster.add_instance(
"clickhouse-client",
)
@pytest.fixture(scope="module")
def started_cluster():
global cluster
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def check_ptr_record(ip, hostname):
try:
host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip)
if hostname.lower() == host.lower():
return True
except socket.herror:
pass
return False
def setup_dns_server(ip):
domains_string = "test3.example.com test2.example.com test1.example.com"
example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com'
run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True)
# DNS server takes time to reload the configuration.
for try_num in range(10):
if all(check_ptr_record(ip, host) for host in domains_string.split()):
break
sleep(1)
def setup_ch_server(dns_server_ip):
ch_server.exec_in_container(
(["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"])
)
ch_server.exec_in_container(
(["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"])
)
ch_server.query("SYSTEM DROP DNS CACHE")
def build_endpoint_v4(ip):
return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'"
def build_endpoint_v6(ip):
return build_endpoint_v4(f"[{ip}]")
def test_host_regexp_multiple_ptr_v4(started_cluster):
server_ip = cluster.get_instance_ip("clickhouse-server")
client_ip = cluster.get_instance_ip("clickhouse-client")
dns_server_ip = cluster.get_instance_ip(cluster.coredns_host)
setup_dns_server(client_ip)
setup_ch_server(dns_server_ip)
current_dir = os.path.dirname(__file__)
client.copy_file_to_container(
os.path.join(current_dir, "scripts", "stress_test.py"), "stress_test.py"
)
client.exec_in_container(["python3", f"stress_test.py", client_ip, server_ip])

View File

@ -2,6 +2,7 @@
This test makes sure interserver cluster queries handle invalid DNS
records for replicas.
"""
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster, ClickHouseInstance

View File

@ -197,7 +197,9 @@ def test_partition_by_string_column(started_cluster):
started_cluster, bucket, "test_foo/bar.csv"
)
assert '3,"йцук"\n' == get_s3_file_content(started_cluster, bucket, "test_йцук.csv")
assert '78,"你好"\n' == get_s3_file_content(started_cluster, bucket, "test_你好.csv")
assert '78,"你好"\n' == get_s3_file_content(
started_cluster, bucket, "test_你好.csv"
)
def test_partition_by_const_column(started_cluster):

View File

@ -1,4 +1,5 @@
"""Test HTTP responses given by the TCP Handler."""
from pathlib import Path
import pytest
from helpers.cluster import ClickHouseCluster

View File

@ -1,4 +1,5 @@
"""Test Interserver responses on configured IP."""
from pathlib import Path
import pytest
from helpers.cluster import ClickHouseCluster

View File

@ -50,7 +50,7 @@ TYPES = {
"UInt32": {"bits": 32, "sign": False, "float": False},
"Int32": {"bits": 32, "sign": True, "float": False},
"UInt64": {"bits": 64, "sign": False, "float": False},
"Int64": {"bits": 64, "sign": True, "float": False}
"Int64": {"bits": 64, "sign": True, "float": False},
# "Float32" : { "bits" : 32, "sign" : True, "float" : True },
# "Float64" : { "bits" : 64, "sign" : True, "float" : True }
}

View File

@ -41,7 +41,7 @@ function thread3()
function thread4()
{
while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table0 FINAL"; done
while true; do $CLICKHOUSE_CLIENT --receive_timeout=3 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done
}
function thread5()

View File

@ -5,20 +5,23 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
DATA_DIR=${CLICKHOUSE_TMP}/data
mkdir -p $TMP_DIR
mkdir -p $DATA_DIR
declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${CURDIR}"
tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${DATA_DIR}"
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS points;
CREATE TABLE points (x Float64, y Float64) ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_size=100000 < "${DATA_DIR}/01037_point_data"
rm "${CURDIR}"/01037_point_data
rm "${DATA_DIR}"/01037_point_data
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS polygons_array;
@ -32,9 +35,9 @@ CREATE TABLE polygons_array
ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_polygon_data"
rm "${CURDIR}"/01037_polygon_data
rm "${DATA_DIR}"/01037_polygon_data
for type in "${SearchTypes[@]}";
do

View File

@ -5,19 +5,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
DATA_DIR=${CLICKHOUSE_TMP}/data
mkdir -p $TMP_DIR
mkdir -p $DATA_DIR
declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}"
tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${DATA_DIR}"
$CLICKHOUSE_CLIENT -n --query="
CREATE TABLE points (x Float64, y Float64) ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_point_data"
rm "${CURDIR}"/01037_point_data
rm "${DATA_DIR}"/01037_point_data
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS polygons_array;
@ -31,9 +34,9 @@ CREATE TABLE polygons_array
ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_polygon_data"
rm "${CURDIR}"/01037_polygon_data
rm "${DATA_DIR}"/01037_polygon_data
for type in "${SearchTypes[@]}";
do

View File

@ -1,104 +1,104 @@
dictGet test_01037.dict_array (-100,-42) qqq 101
dictGet test_01037.dict_array (-1,0) Click South 423
dictGet test_01037.dict_array (-0.1,0) Click South 423
dictGet test_01037.dict_array (0,-2) Click West 424
dictGet test_01037.dict_array (0,-1.1) Click West 424
dictGet test_01037.dict_array (0,1.1) Click North 422
dictGet test_01037.dict_array (0,2) Click North 422
dictGet test_01037.dict_array (0.1,0) Click East 421
dictGet test_01037.dict_array (0.99,2.99) Click North 422
dictGet test_01037.dict_array (1,0) Click East 421
dictGet test_01037.dict_array (3,3) House 314159
dictGet test_01037.dict_array (5,6) Click 42
dictGet test_01037.dict_array (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_array (-100,-42) www 1234
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_array (-100,-42) dd 44
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) ee 55
dictGet test_01037.dict_tuple (-100,-42) qqq 101
dictGet test_01037.dict_tuple (-1,0) Click South 423
dictGet test_01037.dict_tuple (-0.1,0) Click South 423
dictGet test_01037.dict_tuple (0,-2) Click West 424
dictGet test_01037.dict_tuple (0,-1.1) Click West 424
dictGet test_01037.dict_tuple (0,1.1) Click North 422
dictGet test_01037.dict_tuple (0,2) Click North 422
dictGet test_01037.dict_tuple (0.1,0) Click East 421
dictGet test_01037.dict_tuple (0.99,2.99) Click North 422
dictGet test_01037.dict_tuple (1,0) Click East 421
dictGet test_01037.dict_tuple (3,3) House 314159
dictGet test_01037.dict_tuple (5,6) Click 42
dictGet test_01037.dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_tuple (-100,-42) www 1234
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_tuple (-100,-42) dd 44
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) ee 55
dictHas test_01037.dict_array (-100,-42) 0
dictHas test_01037.dict_array (-1,0) 1
dictHas test_01037.dict_array (-0.1,0) 1
dictHas test_01037.dict_array (0,-2) 1
dictHas test_01037.dict_array (0,-1.1) 1
dictHas test_01037.dict_array (0,1.1) 1
dictHas test_01037.dict_array (0,2) 1
dictHas test_01037.dict_array (0.1,0) 1
dictHas test_01037.dict_array (0.99,2.99) 1
dictHas test_01037.dict_array (1,0) 1
dictHas test_01037.dict_array (3,3) 1
dictHas test_01037.dict_array (5,6) 1
dictHas test_01037.dict_array (7.01,7.01) 0
dictHas test_01037.dict_tuple (-100,-42) 0
dictHas test_01037.dict_tuple (-1,0) 1
dictHas test_01037.dict_tuple (-0.1,0) 1
dictHas test_01037.dict_tuple (0,-2) 1
dictHas test_01037.dict_tuple (0,-1.1) 1
dictHas test_01037.dict_tuple (0,1.1) 1
dictHas test_01037.dict_tuple (0,2) 1
dictHas test_01037.dict_tuple (0.1,0) 1
dictHas test_01037.dict_tuple (0.99,2.99) 1
dictHas test_01037.dict_tuple (1,0) 1
dictHas test_01037.dict_tuple (3,3) 1
dictHas test_01037.dict_tuple (5,6) 1
dictHas test_01037.dict_tuple (7.01,7.01) 0
dictGet dict_array (-100,-42) qqq 101
dictGet dict_array (-1,0) Click South 423
dictGet dict_array (-0.1,0) Click South 423
dictGet dict_array (0,-2) Click West 424
dictGet dict_array (0,-1.1) Click West 424
dictGet dict_array (0,1.1) Click North 422
dictGet dict_array (0,2) Click North 422
dictGet dict_array (0.1,0) Click East 421
dictGet dict_array (0.99,2.99) Click North 422
dictGet dict_array (1,0) Click East 421
dictGet dict_array (3,3) House 314159
dictGet dict_array (5,6) Click 42
dictGet dict_array (7.01,7.01) qqq 101
dictGetOrDefault dict_array (-100,-42) www 1234
dictGetOrDefault dict_array (-1,0) Click South 423
dictGetOrDefault dict_array (-0.1,0) Click South 423
dictGetOrDefault dict_array (0,-2) Click West 424
dictGetOrDefault dict_array (0,-1.1) Click West 424
dictGetOrDefault dict_array (0,1.1) Click North 422
dictGetOrDefault dict_array (0,2) Click North 422
dictGetOrDefault dict_array (0.1,0) Click East 421
dictGetOrDefault dict_array (0.99,2.99) Click North 422
dictGetOrDefault dict_array (1,0) Click East 421
dictGetOrDefault dict_array (3,3) House 314159
dictGetOrDefault dict_array (5,6) Click 42
dictGetOrDefault dict_array (7.01,7.01) www 1234
dictGetOrDefault dict_array (-100,-42) dd 44
dictGetOrDefault dict_array (-1,0) Click South 423
dictGetOrDefault dict_array (-0.1,0) Click South 423
dictGetOrDefault dict_array (0,-2) Click West 424
dictGetOrDefault dict_array (0,-1.1) Click West 424
dictGetOrDefault dict_array (0,1.1) Click North 422
dictGetOrDefault dict_array (0,2) Click North 422
dictGetOrDefault dict_array (0.1,0) Click East 421
dictGetOrDefault dict_array (0.99,2.99) Click North 422
dictGetOrDefault dict_array (1,0) Click East 421
dictGetOrDefault dict_array (3,3) House 314159
dictGetOrDefault dict_array (5,6) Click 42
dictGetOrDefault dict_array (7.01,7.01) ee 55
dictGet dict_tuple (-100,-42) qqq 101
dictGet dict_tuple (-1,0) Click South 423
dictGet dict_tuple (-0.1,0) Click South 423
dictGet dict_tuple (0,-2) Click West 424
dictGet dict_tuple (0,-1.1) Click West 424
dictGet dict_tuple (0,1.1) Click North 422
dictGet dict_tuple (0,2) Click North 422
dictGet dict_tuple (0.1,0) Click East 421
dictGet dict_tuple (0.99,2.99) Click North 422
dictGet dict_tuple (1,0) Click East 421
dictGet dict_tuple (3,3) House 314159
dictGet dict_tuple (5,6) Click 42
dictGet dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault dict_tuple (-100,-42) www 1234
dictGetOrDefault dict_tuple (-1,0) Click South 423
dictGetOrDefault dict_tuple (-0.1,0) Click South 423
dictGetOrDefault dict_tuple (0,-2) Click West 424
dictGetOrDefault dict_tuple (0,-1.1) Click West 424
dictGetOrDefault dict_tuple (0,1.1) Click North 422
dictGetOrDefault dict_tuple (0,2) Click North 422
dictGetOrDefault dict_tuple (0.1,0) Click East 421
dictGetOrDefault dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault dict_tuple (1,0) Click East 421
dictGetOrDefault dict_tuple (3,3) House 314159
dictGetOrDefault dict_tuple (5,6) Click 42
dictGetOrDefault dict_tuple (7.01,7.01) www 1234
dictGetOrDefault dict_tuple (-100,-42) dd 44
dictGetOrDefault dict_tuple (-1,0) Click South 423
dictGetOrDefault dict_tuple (-0.1,0) Click South 423
dictGetOrDefault dict_tuple (0,-2) Click West 424
dictGetOrDefault dict_tuple (0,-1.1) Click West 424
dictGetOrDefault dict_tuple (0,1.1) Click North 422
dictGetOrDefault dict_tuple (0,2) Click North 422
dictGetOrDefault dict_tuple (0.1,0) Click East 421
dictGetOrDefault dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault dict_tuple (1,0) Click East 421
dictGetOrDefault dict_tuple (3,3) House 314159
dictGetOrDefault dict_tuple (5,6) Click 42
dictGetOrDefault dict_tuple (7.01,7.01) ee 55
dictHas dict_array (-100,-42) 0
dictHas dict_array (-1,0) 1
dictHas dict_array (-0.1,0) 1
dictHas dict_array (0,-2) 1
dictHas dict_array (0,-1.1) 1
dictHas dict_array (0,1.1) 1
dictHas dict_array (0,2) 1
dictHas dict_array (0.1,0) 1
dictHas dict_array (0.99,2.99) 1
dictHas dict_array (1,0) 1
dictHas dict_array (3,3) 1
dictHas dict_array (5,6) 1
dictHas dict_array (7.01,7.01) 0
dictHas dict_tuple (-100,-42) 0
dictHas dict_tuple (-1,0) 1
dictHas dict_tuple (-0.1,0) 1
dictHas dict_tuple (0,-2) 1
dictHas dict_tuple (0,-1.1) 1
dictHas dict_tuple (0,1.1) 1
dictHas dict_tuple (0,2) 1
dictHas dict_tuple (0.1,0) 1
dictHas dict_tuple (0.99,2.99) 1
dictHas dict_tuple (1,0) 1
dictHas dict_tuple (3,3) 1
dictHas dict_tuple (5,6) 1
dictHas dict_tuple (7.01,7.01) 0

View File

@ -1,56 +1,52 @@
#!/usr/bin/env bash
# Tags: no-debug, no-parallel
# Tags: no-debug
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
mkdir -p $TMP_DIR
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS test_01037;
DROP TABLE IF EXISTS polygons_array;
CREATE DATABASE test_01037;
CREATE TABLE polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO polygons_array VALUES ([[[[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]], [[[5, 5], [5, 1], [7, 1], [7, 7], [1, 7], [1, 5]]]], 'Click', 42);
INSERT INTO polygons_array VALUES ([[[[5, 5], [5, -5], [-5, -5], [-5, 5]], [[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]]], 'House', 314159);
INSERT INTO polygons_array VALUES ([[[[3, 1], [0, 1], [0, -1], [3, -1]]]], 'Click East', 421);
INSERT INTO polygons_array VALUES ([[[[-1, 1], [1, 1], [1, 3], [-1, 3]]]], 'Click North', 422);
INSERT INTO polygons_array VALUES ([[[[-3, 1], [-3, -1], [0, -1], [0, 1]]]], 'Click South', 423);
INSERT INTO polygons_array VALUES ([[[[-1, -1], [1, -1], [1, -3], [-1, -3]]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.polygons_array;
DROP TABLE IF EXISTS polygons_tuple;
CREATE TABLE test_01037.polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_array VALUES ([[[[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]], [[[5, 5], [5, 1], [7, 1], [7, 7], [1, 7], [1, 5]]]], 'Click', 42);
INSERT INTO test_01037.polygons_array VALUES ([[[[5, 5], [5, -5], [-5, -5], [-5, 5]], [[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]]], 'House', 314159);
INSERT INTO test_01037.polygons_array VALUES ([[[[3, 1], [0, 1], [0, -1], [3, -1]]]], 'Click East', 421);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, 1], [1, 1], [1, 3], [-1, 3]]]], 'Click North', 422);
INSERT INTO test_01037.polygons_array VALUES ([[[[-3, 1], [-3, -1], [0, -1], [0, 1]]]], 'Click South', 423);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, -1], [1, -1], [1, -3], [-1, -3]]]], 'Click West', 424);
CREATE TABLE polygons_tuple (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO polygons_tuple VALUES ([[[(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]], [[(5, 5), (5, 1), (7, 1), (7, 7), (1, 7), (1, 5)]]], 'Click', 42);
INSERT INTO polygons_tuple VALUES ([[[(5, 5), (5, -5), (-5, -5), (-5, 5)], [(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]]], 'House', 314159);
INSERT INTO polygons_tuple VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421);
INSERT INTO polygons_tuple VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422);
INSERT INTO polygons_tuple VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423);
INSERT INTO polygons_tuple VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.polygons_tuple;
DROP TABLE IF EXISTS points;
CREATE TABLE test_01037.polygons_tuple (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_tuple VALUES ([[[(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]], [[(5, 5), (5, 1), (7, 1), (7, 7), (1, 7), (1, 5)]]], 'Click', 42);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(5, 5), (5, -5), (-5, -5), (-5, 5)], [(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]]], 'House', 314159);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO test_01037.points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO test_01037.points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO test_01037.points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO test_01037.points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO test_01037.points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO test_01037.points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO test_01037.points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO test_01037.points VALUES (7.01, 7.01, 55, 'ee');
INSERT INTO test_01037.points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO test_01037.points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO test_01037.points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO test_01037.points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO test_01037.points VALUES (0.0, -2.0, 774, 'ffd');
CREATE TABLE points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO points VALUES (7.01, 7.01, 55, 'ee');
INSERT INTO points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO points VALUES (0.0, -2.0, 774, 'ffd');
"
declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
for type in "${SearchTypes[@]}";
@ -58,63 +54,62 @@ do
outputFile="${TMP_DIR}/results${type}.out"
$CLICKHOUSE_CLIENT -n --query="
DROP DICTIONARY IF EXISTS test_01037.dict_array;
CREATE DICTIONARY test_01037.dict_array
DROP DICTIONARY IF EXISTS dict_array;
CREATE DICTIONARY dict_array
(
key Array(Array(Array(Array(Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037'))
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB currentDatabase()))
LIFETIME(0)
LAYOUT($type());
DROP DICTIONARY IF EXISTS test_01037.dict_tuple;
DROP DICTIONARY IF EXISTS dict_tuple;
CREATE DICTIONARY test_01037.dict_tuple
CREATE DICTIONARY dict_tuple
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB 'test_01037'))
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB currentDatabase()))
LIFETIME(0)
LAYOUT($type());
select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
select 'dictGet', 'dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'value', key) from points order by x, y;
select 'dictGetOrDefault', 'dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from points order by x, y;
select 'dictGetOrDefault', 'dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictGet', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, def_i) from points order by x, y;
select 'dictGet', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'value', key) from points order by x, y;
select 'dictGetOrDefault', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from points order by x, y;
select 'dictGetOrDefault', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
dictGetOrDefault(dict_name, 'value', key, def_i) from points order by x, y;
select 'dictHas', 'dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from points order by x, y;
select 'dictHas', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from points order by x, y;
" > "$outputFile"
diff -q "${CURDIR}/01037_polygon_dicts_simple_functions.ans" "$outputFile"
done
$CLICKHOUSE_CLIENT -n --query="
DROP DICTIONARY test_01037.dict_array;
DROP DICTIONARY test_01037.dict_tuple;
DROP TABLE test_01037.polygons_array;
DROP TABLE test_01037.polygons_tuple;
DROP TABLE test_01037.points;
DROP DATABASE test_01037;
DROP DICTIONARY dict_array;
DROP DICTIONARY dict_tuple;
DROP TABLE polygons_array;
DROP TABLE polygons_tuple;
DROP TABLE points;
"

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest
-- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library.
SET optimize_trivial_insert_select=1;
INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION }

View File

@ -85,12 +85,12 @@ c
[4,5,6]
[[1,2],[3,4]]
[[5,6],[7,8]]
0
0
0
0
0
0
1
1
1
1
1
1
1
[2.199219,1.099609,3.300781]
[4.25,3.34961,6.628906]

View File

@ -51,18 +51,15 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/two_dim.npy', Npy, 'value Array(Int8)')"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/three_dim.npy', Npy, 'value Array(Array(Int8))')"
# The test is wrong: https://github.com/ClickHouse/ClickHouse/issues/66458
echo -e "0\n0\n0"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Array(Float32)')" 2>&1 | grep -c "BAD_ARGUMENTS"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value UUID')" 2>&1 | grep -c "BAD_ARGUMENTS"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Tuple(UInt8)')" 2>&1 | grep -c "BAD_ARGUMENTS"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Array(Float32)')" 2>&1 | grep -c "BAD_ARGUMENTS"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value UUID')" 2>&1 | grep -c "UNKNOWN_TYPE"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Tuple(UInt8)')" 2>&1 | grep -c "UNKNOWN_TYPE"
echo -e "0\n0\n0"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS"
#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', Npy, 'value Float32')" 2>&1 | grep -c "BAD_ARGUMENTS"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Int8')" 2>&1 | grep -c "ILLEGAL_COLUMN"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy, 'value Int8')" 2>&1 | grep -c "ILLEGAL_COLUMN"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', Npy, 'value Float32')" 2>&1 | grep -c "ILLEGAL_COLUMN"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/complex.npy')" 2>&1 | grep -c "BAD_ARGUMENTS"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/complex.npy')" 2>&1 | grep -c "CANNOT_EXTRACT_TABLE_STRUCTURE"
$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/float_16.npy')"

View File

@ -6,15 +6,17 @@ drop table if exists dist_out;
create table ephemeral (key Int, value Int) engine=Null();
create table dist_in as ephemeral engine=Distributed(test_shard_localhost, currentDatabase(), ephemeral, key) settings background_insert_batch=1;
create table data (key Int, uniq_values Int) engine=Memory();
create materialized view mv to data as select key, uniqExact(value) uniq_values from ephemeral group by key;
create table data (key Int, uniq_values Int) engine=TinyLog();
create materialized view mv to data as select key, uniqExact(value::String) uniq_values from ephemeral group by key;
system stop distributed sends dist_in;
create table dist_out as data engine=Distributed(test_shard_localhost, currentDatabase(), data);
set prefer_localhost_replica=0;
SET optimize_trivial_insert_select = 1;
insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi';
-- due to pushing to MV with aggregation the query needs ~300MiB
-- but it will be done in background via "system flush distributed"
insert into dist_in select number/100, number from system.numbers limit 3e6 settings max_block_size=3e6, max_memory_usage='100Mi';
system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED }
system flush distributed dist_in settings max_memory_usage=0;
select count() from dist_out;

View File

@ -0,0 +1 @@
Hello, world!

View File

@ -0,0 +1,20 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --multiquery "
DROP TABLE IF EXISTS test;
CREATE TABLE test (s String) ORDER BY ();
INSERT INTO test VALUES ('Hello, world!');
"
${CLICKHOUSE_LOCAL} --multiquery "
CREATE NAMED COLLECTION mydb AS host = '${CLICKHOUSE_HOST}', port = ${CLICKHOUSE_PORT_TCP}, user = 'default', password = '', db = '${CLICKHOUSE_DATABASE}';
SELECT * FROM remote(mydb, table = 'test');
" 2>&1 | grep --text -F -v "ASan doesn't fully support makecontext/swapcontext functions"
${CLICKHOUSE_CLIENT} --multiquery "
DROP TABLE test;
"

View File

@ -0,0 +1 @@
ok

View File

@ -0,0 +1,14 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so the grpc library is not built
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
if [[ -z "$CLICKHOUSE_GRPC_CLIENT" ]]; then
CLICKHOUSE_GRPC_CLIENT="$CURDIR/../../../utils/grpc-client/clickhouse-grpc-client.py"
fi
# Simple test.
$CLICKHOUSE_GRPC_CLIENT --query "SELECT 'ok'"

View File

@ -0,0 +1,52 @@
#!/usr/bin/env python3
# This is a helper utility.
# It generates files in the "pb2" folder using the protocol buffer compiler.
# This script must be called manually after any change pf "clickhouse_grpc.proto"
import grpc_tools # pip3 install grpcio-tools
import os, shutil, subprocess
# Settings.
script_path = os.path.realpath(__file__)
script_name = os.path.basename(script_path)
script_dir = os.path.dirname(script_path)
root_dir = os.path.abspath(os.path.join(script_dir, "../.."))
grpc_proto_dir = os.path.abspath(os.path.join(root_dir, "src/Server/grpc_protos"))
grpc_proto_filename = "clickhouse_grpc.proto"
# Files in the "pb2" folder which will be generated by this script.
pb2_filenames = ["clickhouse_grpc_pb2.py", "clickhouse_grpc_pb2_grpc.py"]
pb2_dir = os.path.join(script_dir, "pb2")
# Processes the protobuf schema with the protocol buffer compiler and generates the "pb2" folder.
def generate_pb2():
print(f"Generating files:")
for pb2_filename in pb2_filenames:
print(os.path.join(pb2_dir, pb2_filename))
os.makedirs(pb2_dir, exist_ok=True)
cmd = [
"python3",
"-m",
"grpc_tools.protoc",
"-I" + grpc_proto_dir,
"--python_out=" + pb2_dir,
"--grpc_python_out=" + pb2_dir,
os.path.join(grpc_proto_dir, grpc_proto_filename),
]
subprocess.run(cmd)
for pb2_filename in pb2_filenames:
assert os.path.exists(os.path.join(pb2_dir, pb2_filename))
print("Done! (generate_pb2)")
# MAIN
if __name__ == "__main__":
generate_pb2()

View File

@ -1,29 +0,0 @@
#!/usr/bin/env python3
import grpc_tools # pip3 install grpcio-tools
import os
import subprocess
script_dir = os.path.dirname(os.path.realpath(__file__))
dest_dir = script_dir
src_dir = os.path.abspath(os.path.join(script_dir, "../../../src/Server/grpc_protos"))
src_filename = "clickhouse_grpc.proto"
def generate():
cmd = [
"python3",
"-m",
"grpc_tools.protoc",
"-I" + src_dir,
"--python_out=" + dest_dir,
"--grpc_python_out=" + dest_dir,
os.path.join(src_dir, src_filename),
]
subprocess.run(cmd)
if __name__ == "__main__":
generate()