diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 74de1a3e9bd..91036d88d8c 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -4,14 +4,26 @@ FROM ubuntu:20.04 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ - && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ + && apt-get install \ + apt-transport-https \ + apt-utils \ + ca-certificates \ + dnsutils \ + gnupg \ + iputils-ping \ + lsb-release \ + wget \ --yes --no-install-recommends --verbose-versions \ + && cat /etc/resolv.conf \ + && echo "nameserver 1.1.1.1" >> /etc/resolv.conf \ + && nslookup -debug apt.llvm.org \ + && ping -c1 apt.llvm.org \ + && wget -nv --retry-connrefused --tries=10 -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ + && echo "deb [trusted=yes] https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ /etc/apt/sources.list # initial packages @@ -24,7 +36,10 @@ RUN apt-get update \ software-properties-common \ --yes --no-install-recommends -RUN apt-get update \ +RUN cat /etc/resolv.conf \ + && echo "nameserver 1.1.1.1" >> /etc/resolv.conf \ + && nslookup -debug apt.llvm.org \ + && apt-get update \ && apt-get install \ bash \ cmake \ diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index e8653c2122e..158d2608f41 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -4,8 +4,9 @@ FROM ubuntu:20.04 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ - && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ + && apt-get install apt-utils ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ + && echo "nameserver 1.1.1.1" >> /etc/resolv.conf \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ @@ -31,7 +32,8 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb \ && chmod +x dpkg-deb \ && cp dpkg-deb /usr/bin -RUN apt-get update \ +RUN echo "nameserver 1.1.1.1" >> /etc/resolv.conf \ + && apt-get update \ && apt-get install \ clang-${LLVM_VERSION} \ debhelper \ diff --git a/docker/test/sqlancer/process_sqlancer_result.py b/docker/test/sqlancer/process_sqlancer_result.py index 411c1e18e19..ede3cabc1c5 100755 --- a/docker/test/sqlancer/process_sqlancer_result.py +++ b/docker/test/sqlancer/process_sqlancer_result.py @@ -26,6 +26,7 @@ def process_result(result_folder): with open(err_path, 'r') as f: if 'AssertionError' in f.read(): summary.append((test, "FAIL")) + status = 'failure' else: summary.append((test, "OK")) diff --git a/docker/test/sqlancer/run.sh b/docker/test/sqlancer/run.sh index 20e82603567..e465ba1c993 100755 --- a/docker/test/sqlancer/run.sh +++ b/docker/test/sqlancer/run.sh @@ -11,7 +11,7 @@ service clickhouse-server start && sleep 5 cd /sqlancer/sqlancer-master -export TIMEOUT=60 +export TIMEOUT=300 export NUM_QUERIES=1000 ( java -jar target/sqlancer-*.jar --num-threads 10 --timeout-seconds $TIMEOUT --num-queries $NUM_QUERIES --username default --password "" clickhouse --oracle TLPWhere | tee /test_output/TLPWhere.out ) 3>&1 1>&2 2>&3 | tee /test_output/TLPWhere.err diff --git a/src/Common/ya.make b/src/Common/ya.make index debad6c5de2..d1ff04f8f0a 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -14,7 +14,7 @@ PEERDIR( clickhouse/base/common clickhouse/base/pcg-random clickhouse/base/widechar_width - contrib/libs/libcpuid/libcpuid + contrib/libs/libcpuid contrib/libs/openssl contrib/libs/poco/NetSSL_OpenSSL contrib/libs/re2 diff --git a/src/Common/ya.make.in b/src/Common/ya.make.in index 210ecab6ef6..420384bb4a9 100644 --- a/src/Common/ya.make.in +++ b/src/Common/ya.make.in @@ -13,7 +13,7 @@ PEERDIR( clickhouse/base/common clickhouse/base/pcg-random clickhouse/base/widechar_width - contrib/libs/libcpuid/libcpuid + contrib/libs/libcpuid contrib/libs/openssl contrib/libs/poco/NetSSL_OpenSSL contrib/libs/re2 diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 47a0e0655d6..7586b4a28f0 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -209,7 +209,13 @@ void DiskCacheWrapper::clearDirectory(const String & path) void DiskCacheWrapper::moveDirectory(const String & from_path, const String & to_path) { if (cache_disk->exists(from_path)) + { + /// Destination directory may not be empty if previous directory move attempt was failed. + if (cache_disk->exists(to_path) && cache_disk->isDirectory(to_path)) + cache_disk->clearDirectory(to_path); + cache_disk->moveDirectory(from_path, to_path); + } DiskDecorator::moveDirectory(from_path, to_path); } diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 2b0082f4334..95b9e25ead1 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -40,7 +40,8 @@ struct AddSecondsImpl { static constexpr auto name = "addSeconds"; - static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) + static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) { return {t.whole + delta, t.fractional}; } @@ -60,7 +61,8 @@ struct AddMinutesImpl { static constexpr auto name = "addMinutes"; - static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) + static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) { return {t.whole + delta * 60, t.fractional}; } @@ -80,7 +82,8 @@ struct AddHoursImpl { static constexpr auto name = "addHours"; - static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) + static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl &) { return {t.whole + delta * 3600, t.fractional}; } @@ -99,7 +102,8 @@ struct AddDaysImpl { static constexpr auto name = "addDays"; - static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) + static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) { return {time_zone.addDays(t.whole, delta), t.fractional}; } @@ -119,7 +123,8 @@ struct AddWeeksImpl { static constexpr auto name = "addWeeks"; - static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) + static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) { return {time_zone.addWeeks(t.whole, delta), t.fractional}; } @@ -139,7 +144,8 @@ struct AddMonthsImpl { static constexpr auto name = "addMonths"; - static inline DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) + static inline DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) { return {time_zone.addMonths(t.whole, delta), t.fractional}; } @@ -159,7 +165,8 @@ struct AddQuartersImpl { static constexpr auto name = "addQuarters"; - static inline DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) + static inline DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) { return {time_zone.addQuarters(t.whole, delta), t.fractional}; } @@ -179,7 +186,8 @@ struct AddYearsImpl { static constexpr auto name = "addYears"; - static inline DecimalUtils::DecimalComponents execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) + static inline DecimalUtils::DecimalComponents + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone) { return {time_zone.addYears(t.whole, delta), t.fractional}; } @@ -265,14 +273,16 @@ struct Adder private: template - NO_INLINE NO_SANITIZE_UNDEFINED void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const + NO_INLINE NO_SANITIZE_UNDEFINED void vectorVector( + const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const { for (size_t i = 0; i < size; ++i) vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone); } template - NO_INLINE NO_SANITIZE_UNDEFINED void constantVector(const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const + NO_INLINE NO_SANITIZE_UNDEFINED void constantVector( + const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const { for (size_t i = 0; i < size; ++i) vec_to[i] = transform.execute(from, delta.getData()[i], time_zone); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index ef74f7778b1..5e3831b37a2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2798,12 +2798,16 @@ private: auto & out_data = static_cast(*res).getData(); out_data.resize(size); + auto default_enum_value = result_type.getValues().front().second; + if (nullable_col) { for (const auto i : ext::range(0, size)) { if (!nullable_col->isNullAt(i)) out_data[i] = result_type.getValue(col->getDataAt(i)); + else + out_data[i] = default_enum_value; } } else diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index e42c3155327..4eab2a491c7 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -49,8 +49,11 @@ public: {} template - inline auto execute(const DateTime64 & t, Args && ... args) const + inline auto NO_SANITIZE_UNDEFINED execute(const DateTime64 & t, Args && ... args) const { + /// Type conversion from float to integer may be required. + /// We are Ok with implementation specific result for out of range and denormals conversion. + if constexpr (TransformHasExecuteOverload_v) { return wrapped_transform.execute(t, scale_multiplier, std::forward(args)...); diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index 55396b1d1c7..804696f2776 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -49,11 +49,10 @@ struct DivideIntegralByConstantImpl #pragma GCC diagnostic ignored "-Wsign-compare" /// Division by -1. By the way, we avoid FPE by division of the largest negative number by -1. - /// And signed integer overflow is well defined in C++20. if (unlikely(is_signed_v && b == -1)) { for (size_t i = 0; i < size; ++i) - c_pos[i] = -a_pos[i]; + c_pos[i] = -make_unsigned_t(a_pos[i]); /// Avoid UBSan report in signed integer overflow. return; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 925bd912d42..7373b24991a 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -21,6 +21,7 @@ namespace ProfileEvents extern const Event S3WriteBytes; } + namespace DB { // S3 protocol does not allow to have multipart upload with more than 10000 parts. @@ -50,9 +51,9 @@ WriteBufferFromS3::WriteBufferFromS3( , client_ptr(std::move(client_ptr_)) , minimum_upload_part_size(minimum_upload_part_size_) , max_single_part_upload_size(max_single_part_upload_size_) - , temporary_buffer(Aws::MakeShared("temporary buffer")) - , last_part_size(0) -{ } +{ + allocateBuffer(); +} void WriteBufferFromS3::nextImpl() { @@ -72,11 +73,17 @@ void WriteBufferFromS3::nextImpl() if (!multipart_upload_id.empty() && last_part_size > minimum_upload_part_size) { writePart(); - last_part_size = 0; - temporary_buffer = Aws::MakeShared("temporary buffer"); + allocateBuffer(); } } +void WriteBufferFromS3::allocateBuffer() +{ + temporary_buffer = Aws::MakeShared("temporary buffer"); + temporary_buffer->exceptions(std::ios::badbit); + last_part_size = 0; +} + void WriteBufferFromS3::finalize() { /// FIXME move final flush into the caller @@ -130,17 +137,26 @@ void WriteBufferFromS3::createMultipartUpload() if (outcome.IsSuccess()) { multipart_upload_id = outcome.GetResult().GetUploadId(); - LOG_DEBUG(log, "Multipart upload has created. Upload id: {}", multipart_upload_id); + LOG_DEBUG(log, "Multipart upload has created. Bucket: {}, Key: {}, Upload id: {}", bucket, key, multipart_upload_id); } else throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); } - void WriteBufferFromS3::writePart() { - if (temporary_buffer->tellp() <= 0) + auto size = temporary_buffer->tellp(); + + LOG_DEBUG(log, "Writing part. Bucket: {}, Key: {}, Upload_id: {}, Size: {}", bucket, key, multipart_upload_id, size); + + if (size < 0) + throw Exception("Failed to write part. Buffer in invalid state.", ErrorCodes::S3_ERROR); + + if (size == 0) + { + LOG_DEBUG(log, "Skipping writing part. Buffer is empty."); return; + } if (part_tags.size() == S3_WARN_MAX_PARTS) { @@ -154,19 +170,16 @@ void WriteBufferFromS3::writePart() req.SetKey(key); req.SetPartNumber(part_tags.size() + 1); req.SetUploadId(multipart_upload_id); - req.SetContentLength(temporary_buffer->tellp()); + req.SetContentLength(size); req.SetBody(temporary_buffer); auto outcome = client_ptr->UploadPart(req); - LOG_TRACE( - log, "Writing part. Bucket: {}, Key: {}, Upload_id: {}, Data size: {}", bucket, key, multipart_upload_id, req.GetContentLength()); - if (outcome.IsSuccess()) { auto etag = outcome.GetResult().GetETag(); part_tags.push_back(etag); - LOG_DEBUG(log, "Writing part finished. Total parts: {}, Upload_id: {}, Etag: {}", part_tags.size(), multipart_upload_id, etag); + LOG_DEBUG(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, etag, part_tags.size()); } else throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); @@ -174,7 +187,10 @@ void WriteBufferFromS3::writePart() void WriteBufferFromS3::completeMultipartUpload() { - LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, multipart_upload_id); + LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, part_tags.size()); + + if (part_tags.empty()) + throw Exception("Failed to complete multipart upload. No parts have uploaded", ErrorCodes::S3_ERROR); Aws::S3::Model::CompleteMultipartUploadRequest req; req.SetBucket(bucket); @@ -193,22 +209,30 @@ void WriteBufferFromS3::completeMultipartUpload() auto outcome = client_ptr->CompleteMultipartUpload(req); if (outcome.IsSuccess()) - LOG_DEBUG(log, "Multipart upload has completed. Upload_id: {}", multipart_upload_id); + LOG_DEBUG(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, part_tags.size()); else throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); } void WriteBufferFromS3::makeSinglepartUpload() { - if (temporary_buffer->tellp() <= 0) - return; + auto size = temporary_buffer->tellp(); - LOG_DEBUG(log, "Making single part upload. Bucket: {}, Key: {}", bucket, key); + LOG_DEBUG(log, "Making single part upload. Bucket: {}, Key: {}, Size: {}", bucket, key, size); + + if (size < 0) + throw Exception("Failed to make single part upload. Buffer in invalid state", ErrorCodes::S3_ERROR); + + if (size == 0) + { + LOG_DEBUG(log, "Skipping single part upload. Buffer is empty."); + return; + } Aws::S3::Model::PutObjectRequest req; req.SetBucket(bucket); req.SetKey(key); - req.SetContentLength(temporary_buffer->tellp()); + req.SetContentLength(size); req.SetBody(temporary_buffer); if (object_metadata.has_value()) req.SetMetadata(object_metadata.value()); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 9e4b056603a..7a77949072b 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -69,6 +69,8 @@ public: private: bool finalized = false; + void allocateBuffer(); + void createMultipartUpload(); void writePart(); void completeMultipartUpload(); diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 42e96bae07b..db19c000cfd 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -290,8 +290,6 @@ std::optional evaluateExpressionOverConstantCondition(const ASTPtr & nod { Blocks result; - // TODO: `node` may be always-false literal. - if (const auto * fn = node->as()) { const auto dnf = analyzeFunction(fn, target_expr); @@ -350,6 +348,14 @@ std::optional evaluateExpressionOverConstantCondition(const ASTPtr & nod } } } + else if (const auto * literal = node->as()) + { + // Check if it's always true or false. + if (literal->value.getType() == Field::Types::UInt64 && literal->value.get() == 0) + return {result}; + else + return {}; + } return {result}; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1f18c894465..7f2a9cdb1f6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1013,7 +1013,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); - volume->getDisk()->moveFile(from, to); + volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; SyncGuardPtr sync_guard; @@ -1065,7 +1065,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const try { - volume->getDisk()->moveFile(from, to); + volume->getDisk()->moveDirectory(from, to); } catch (const Poco::FileNotFoundException &) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a16430c20a4..2c4003919f4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1389,7 +1389,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const DataPartsLock lock(mutex); for (MutableDataPartPtr & part : dst_parts) - dest_table_storage->renameTempPartAndReplace(part, &increment, &transaction, lock); + dest_table_storage->renameTempPartAndReplace(part, &dest_table_storage->increment, &transaction, lock); removePartsFromWorkingSet(src_parts, true, lock); transaction.commit(&lock); diff --git a/tests/queries/0_stateless/01755_shard_pruning_with_literal.reference b/tests/queries/0_stateless/01755_shard_pruning_with_literal.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/01755_shard_pruning_with_literal.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/01755_shard_pruning_with_literal.sql b/tests/queries/0_stateless/01755_shard_pruning_with_literal.sql new file mode 100644 index 00000000000..0e93d76573c --- /dev/null +++ b/tests/queries/0_stateless/01755_shard_pruning_with_literal.sql @@ -0,0 +1,14 @@ +set optimize_skip_unused_shards=1; + +drop table if exists data_01755; +drop table if exists dist_01755; + +create table data_01755 (i Int) Engine=Memory; +create table dist_01755 as data_01755 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01755, i); + +insert into data_01755 values (1); + +select * from dist_01755 where 1 settings enable_early_constant_folding = 0; + +drop table if exists data_01755; +drop table if exists dist_01755; diff --git a/tests/queries/0_stateless/01761_cast_to_enum_nullable.reference b/tests/queries/0_stateless/01761_cast_to_enum_nullable.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01761_cast_to_enum_nullable.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01761_cast_to_enum_nullable.sql b/tests/queries/0_stateless/01761_cast_to_enum_nullable.sql new file mode 100644 index 00000000000..42a51d2f7b9 --- /dev/null +++ b/tests/queries/0_stateless/01761_cast_to_enum_nullable.sql @@ -0,0 +1 @@ +SELECT toUInt8(assumeNotNull(cast(cast(NULL, 'Nullable(String)'), 'Nullable(Enum8(\'Hello\' = 1))'))); diff --git a/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.reference b/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.reference new file mode 100644 index 00000000000..a07ed155918 --- /dev/null +++ b/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.reference @@ -0,0 +1,4 @@ +1 1 1_1_1_0 +1 2 1_2_2_0 +1 3 1_3_3_0 +1 4 1_4_4_0 diff --git a/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.sql b/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.sql new file mode 100644 index 00000000000..ea00c573c74 --- /dev/null +++ b/tests/queries/0_stateless/01765_move_to_table_overlapping_block_number.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t_src; +DROP TABLE IF EXISTS t_dst; + +CREATE TABLE t_src (id UInt32, v UInt32) ENGINE = MergeTree ORDER BY id PARTITION BY id; +CREATE TABLE t_dst (id UInt32, v UInt32) ENGINE = MergeTree ORDER BY id PARTITION BY id; + +SYSTEM STOP MERGES t_src; +SYSTEM STOP MERGES t_dst; + +INSERT INTO t_dst VALUES (1, 1); +INSERT INTO t_dst VALUES (1, 2); +INSERT INTO t_dst VALUES (1, 3); + +INSERT INTO t_src VALUES (1, 4); + +ALTER TABLE t_src MOVE PARTITION 1 TO TABLE t_dst; +SELECT *, _part FROM t_dst ORDER BY v; + +DROP TABLE t_src; +DROP TABLE t_dst; diff --git a/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.reference b/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.reference new file mode 100644 index 00000000000..6b764d18a4d --- /dev/null +++ b/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.reference @@ -0,0 +1,10 @@ +-9223372036854775807 +-9223372036854775808 +9223372036854775807 +9223372036854775806 +9223372036854775805 +9223372036854775804 +9223372036854775803 +9223372036854775802 +9223372036854775801 +9223372036854775800 diff --git a/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.sql b/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.sql new file mode 100644 index 00000000000..20b4f585182 --- /dev/null +++ b/tests/queries/0_stateless/01772_intdiv_minus_one_ubsan.sql @@ -0,0 +1 @@ +SELECT intDiv(toInt64(number), -1) FROM numbers(9223372036854775807, 10); diff --git a/tests/queries/0_stateless/01773_datetime64_add_ubsan.reference b/tests/queries/0_stateless/01773_datetime64_add_ubsan.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/01773_datetime64_add_ubsan.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/01773_datetime64_add_ubsan.sql b/tests/queries/0_stateless/01773_datetime64_add_ubsan.sql new file mode 100644 index 00000000000..f7267f2b6b4 --- /dev/null +++ b/tests/queries/0_stateless/01773_datetime64_add_ubsan.sql @@ -0,0 +1,2 @@ +-- The result is unspecified but UBSan should not argue. +SELECT ignore(addHours(now64(3), inf)) FROM numbers(2); diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index d38b34f3419..d534fd8fd4f 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -32,7 +32,10 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (db-generator) add_subdirectory (wal-dump) add_subdirectory (check-mysql-binlog) - add_subdirectory (memcpy-bench) + + if (NOT OS_DARWIN) + add_subdirectory (memcpy-bench) + endif () endif () if (ENABLE_CODE_QUALITY)