Merge branch 'master' into to-start-of-interval-hour-align

This commit is contained in:
Alexey Milovidov 2021-03-24 22:46:06 +03:00
commit 67395aad69
27 changed files with 179 additions and 47 deletions

View File

@ -4,14 +4,26 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN apt-get update \ 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 \ --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" \ && 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 \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \
&& apt-key add /tmp/llvm-snapshot.gpg.key \ && apt-key add /tmp/llvm-snapshot.gpg.key \
&& export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ && 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 /etc/apt/sources.list
# initial packages # initial packages
@ -24,7 +36,10 @@ RUN apt-get update \
software-properties-common \ software-properties-common \
--yes --no-install-recommends --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 \ && apt-get install \
bash \ bash \
cmake \ cmake \

View File

@ -4,8 +4,9 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN apt-get update \ 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 \ --yes --no-install-recommends --verbose-versions \
&& echo "nameserver 1.1.1.1" >> /etc/resolv.conf \
&& export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \
&& wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && 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 \ && 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 \ && chmod +x dpkg-deb \
&& cp dpkg-deb /usr/bin && 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 \ && apt-get install \
clang-${LLVM_VERSION} \ clang-${LLVM_VERSION} \
debhelper \ debhelper \

View File

@ -26,6 +26,7 @@ def process_result(result_folder):
with open(err_path, 'r') as f: with open(err_path, 'r') as f:
if 'AssertionError' in f.read(): if 'AssertionError' in f.read():
summary.append((test, "FAIL")) summary.append((test, "FAIL"))
status = 'failure'
else: else:
summary.append((test, "OK")) summary.append((test, "OK"))

View File

@ -11,7 +11,7 @@ service clickhouse-server start && sleep 5
cd /sqlancer/sqlancer-master cd /sqlancer/sqlancer-master
export TIMEOUT=60 export TIMEOUT=300
export NUM_QUERIES=1000 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 ( 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

View File

@ -14,7 +14,7 @@ PEERDIR(
clickhouse/base/common clickhouse/base/common
clickhouse/base/pcg-random clickhouse/base/pcg-random
clickhouse/base/widechar_width clickhouse/base/widechar_width
contrib/libs/libcpuid/libcpuid contrib/libs/libcpuid
contrib/libs/openssl contrib/libs/openssl
contrib/libs/poco/NetSSL_OpenSSL contrib/libs/poco/NetSSL_OpenSSL
contrib/libs/re2 contrib/libs/re2

View File

@ -13,7 +13,7 @@ PEERDIR(
clickhouse/base/common clickhouse/base/common
clickhouse/base/pcg-random clickhouse/base/pcg-random
clickhouse/base/widechar_width clickhouse/base/widechar_width
contrib/libs/libcpuid/libcpuid contrib/libs/libcpuid
contrib/libs/openssl contrib/libs/openssl
contrib/libs/poco/NetSSL_OpenSSL contrib/libs/poco/NetSSL_OpenSSL
contrib/libs/re2 contrib/libs/re2

View File

@ -209,7 +209,13 @@ void DiskCacheWrapper::clearDirectory(const String & path)
void DiskCacheWrapper::moveDirectory(const String & from_path, const String & to_path) void DiskCacheWrapper::moveDirectory(const String & from_path, const String & to_path)
{ {
if (cache_disk->exists(from_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); cache_disk->moveDirectory(from_path, to_path);
}
DiskDecorator::moveDirectory(from_path, to_path); DiskDecorator::moveDirectory(from_path, to_path);
} }

View File

@ -40,7 +40,8 @@ struct AddSecondsImpl
{ {
static constexpr auto name = "addSeconds"; static constexpr auto name = "addSeconds";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &) static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
{ {
return {t.whole + delta, t.fractional}; return {t.whole + delta, t.fractional};
} }
@ -60,7 +61,8 @@ struct AddMinutesImpl
{ {
static constexpr auto name = "addMinutes"; static constexpr auto name = "addMinutes";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &) static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
{ {
return {t.whole + delta * 60, t.fractional}; return {t.whole + delta * 60, t.fractional};
} }
@ -80,7 +82,8 @@ struct AddHoursImpl
{ {
static constexpr auto name = "addHours"; static constexpr auto name = "addHours";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &) static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
{ {
return {t.whole + delta * 3600, t.fractional}; return {t.whole + delta * 3600, t.fractional};
} }
@ -99,7 +102,8 @@ struct AddDaysImpl
{ {
static constexpr auto name = "addDays"; static constexpr auto name = "addDays";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone) static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
{ {
return {time_zone.addDays(t.whole, delta), t.fractional}; return {time_zone.addDays(t.whole, delta), t.fractional};
} }
@ -119,7 +123,8 @@ struct AddWeeksImpl
{ {
static constexpr auto name = "addWeeks"; static constexpr auto name = "addWeeks";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone) static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
{ {
return {time_zone.addWeeks(t.whole, delta), t.fractional}; return {time_zone.addWeeks(t.whole, delta), t.fractional};
} }
@ -139,7 +144,8 @@ struct AddMonthsImpl
{ {
static constexpr auto name = "addMonths"; static constexpr auto name = "addMonths";
static inline DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone) static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
{ {
return {time_zone.addMonths(t.whole, delta), t.fractional}; return {time_zone.addMonths(t.whole, delta), t.fractional};
} }
@ -159,7 +165,8 @@ struct AddQuartersImpl
{ {
static constexpr auto name = "addQuarters"; static constexpr auto name = "addQuarters";
static inline DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone) static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
{ {
return {time_zone.addQuarters(t.whole, delta), t.fractional}; return {time_zone.addQuarters(t.whole, delta), t.fractional};
} }
@ -179,7 +186,8 @@ struct AddYearsImpl
{ {
static constexpr auto name = "addYears"; static constexpr auto name = "addYears";
static inline DecimalUtils::DecimalComponents<DateTime64> execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone) static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
{ {
return {time_zone.addYears(t.whole, delta), t.fractional}; return {time_zone.addYears(t.whole, delta), t.fractional};
} }
@ -265,14 +273,16 @@ struct Adder
private: private:
template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType> template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType>
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) for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone); vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone);
} }
template <typename FromType, typename ToVectorType, typename DeltaColumnType> template <typename FromType, typename ToVectorType, typename DeltaColumnType>
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) for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, delta.getData()[i], time_zone); vec_to[i] = transform.execute(from, delta.getData()[i], time_zone);

View File

@ -2798,12 +2798,16 @@ private:
auto & out_data = static_cast<typename EnumType::ColumnType &>(*res).getData(); auto & out_data = static_cast<typename EnumType::ColumnType &>(*res).getData();
out_data.resize(size); out_data.resize(size);
auto default_enum_value = result_type.getValues().front().second;
if (nullable_col) if (nullable_col)
{ {
for (const auto i : ext::range(0, size)) for (const auto i : ext::range(0, size))
{ {
if (!nullable_col->isNullAt(i)) if (!nullable_col->isNullAt(i))
out_data[i] = result_type.getValue(col->getDataAt(i)); out_data[i] = result_type.getValue(col->getDataAt(i));
else
out_data[i] = default_enum_value;
} }
} }
else else

View File

@ -49,8 +49,11 @@ public:
{} {}
template <typename ... Args> template <typename ... Args>
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<DateTime64, decltype(scale_multiplier), Args...>) if constexpr (TransformHasExecuteOverload_v<DateTime64, decltype(scale_multiplier), Args...>)
{ {
return wrapped_transform.execute(t, scale_multiplier, std::forward<Args>(args)...); return wrapped_transform.execute(t, scale_multiplier, std::forward<Args>(args)...);

View File

@ -49,11 +49,10 @@ struct DivideIntegralByConstantImpl
#pragma GCC diagnostic ignored "-Wsign-compare" #pragma GCC diagnostic ignored "-Wsign-compare"
/// Division by -1. By the way, we avoid FPE by division of the largest negative number by -1. /// 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> && b == -1)) if (unlikely(is_signed_v<B> && b == -1))
{ {
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
c_pos[i] = -a_pos[i]; c_pos[i] = -make_unsigned_t<A>(a_pos[i]); /// Avoid UBSan report in signed integer overflow.
return; return;
} }

View File

@ -21,6 +21,7 @@ namespace ProfileEvents
extern const Event S3WriteBytes; extern const Event S3WriteBytes;
} }
namespace DB namespace DB
{ {
// S3 protocol does not allow to have multipart upload with more than 10000 parts. // 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_)) , client_ptr(std::move(client_ptr_))
, minimum_upload_part_size(minimum_upload_part_size_) , minimum_upload_part_size(minimum_upload_part_size_)
, max_single_part_upload_size(max_single_part_upload_size_) , max_single_part_upload_size(max_single_part_upload_size_)
, temporary_buffer(Aws::MakeShared<Aws::StringStream>("temporary buffer")) {
, last_part_size(0) allocateBuffer();
{ } }
void WriteBufferFromS3::nextImpl() void WriteBufferFromS3::nextImpl()
{ {
@ -72,11 +73,17 @@ void WriteBufferFromS3::nextImpl()
if (!multipart_upload_id.empty() && last_part_size > minimum_upload_part_size) if (!multipart_upload_id.empty() && last_part_size > minimum_upload_part_size)
{ {
writePart(); writePart();
last_part_size = 0; allocateBuffer();
temporary_buffer = Aws::MakeShared<Aws::StringStream>("temporary buffer");
} }
} }
void WriteBufferFromS3::allocateBuffer()
{
temporary_buffer = Aws::MakeShared<Aws::StringStream>("temporary buffer");
temporary_buffer->exceptions(std::ios::badbit);
last_part_size = 0;
}
void WriteBufferFromS3::finalize() void WriteBufferFromS3::finalize()
{ {
/// FIXME move final flush into the caller /// FIXME move final flush into the caller
@ -130,17 +137,26 @@ void WriteBufferFromS3::createMultipartUpload()
if (outcome.IsSuccess()) if (outcome.IsSuccess())
{ {
multipart_upload_id = outcome.GetResult().GetUploadId(); 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 else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
} }
void WriteBufferFromS3::writePart() 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; return;
}
if (part_tags.size() == S3_WARN_MAX_PARTS) if (part_tags.size() == S3_WARN_MAX_PARTS)
{ {
@ -154,19 +170,16 @@ void WriteBufferFromS3::writePart()
req.SetKey(key); req.SetKey(key);
req.SetPartNumber(part_tags.size() + 1); req.SetPartNumber(part_tags.size() + 1);
req.SetUploadId(multipart_upload_id); req.SetUploadId(multipart_upload_id);
req.SetContentLength(temporary_buffer->tellp()); req.SetContentLength(size);
req.SetBody(temporary_buffer); req.SetBody(temporary_buffer);
auto outcome = client_ptr->UploadPart(req); 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()) if (outcome.IsSuccess())
{ {
auto etag = outcome.GetResult().GetETag(); auto etag = outcome.GetResult().GetETag();
part_tags.push_back(etag); 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 else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
@ -174,7 +187,10 @@ void WriteBufferFromS3::writePart()
void WriteBufferFromS3::completeMultipartUpload() 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; Aws::S3::Model::CompleteMultipartUploadRequest req;
req.SetBucket(bucket); req.SetBucket(bucket);
@ -193,22 +209,30 @@ void WriteBufferFromS3::completeMultipartUpload()
auto outcome = client_ptr->CompleteMultipartUpload(req); auto outcome = client_ptr->CompleteMultipartUpload(req);
if (outcome.IsSuccess()) 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 else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
} }
void WriteBufferFromS3::makeSinglepartUpload() void WriteBufferFromS3::makeSinglepartUpload()
{ {
if (temporary_buffer->tellp() <= 0) auto size = temporary_buffer->tellp();
return;
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; Aws::S3::Model::PutObjectRequest req;
req.SetBucket(bucket); req.SetBucket(bucket);
req.SetKey(key); req.SetKey(key);
req.SetContentLength(temporary_buffer->tellp()); req.SetContentLength(size);
req.SetBody(temporary_buffer); req.SetBody(temporary_buffer);
if (object_metadata.has_value()) if (object_metadata.has_value())
req.SetMetadata(object_metadata.value()); req.SetMetadata(object_metadata.value());

View File

@ -69,6 +69,8 @@ public:
private: private:
bool finalized = false; bool finalized = false;
void allocateBuffer();
void createMultipartUpload(); void createMultipartUpload();
void writePart(); void writePart();
void completeMultipartUpload(); void completeMultipartUpload();

View File

@ -290,8 +290,6 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
{ {
Blocks result; Blocks result;
// TODO: `node` may be always-false literal.
if (const auto * fn = node->as<ASTFunction>()) if (const auto * fn = node->as<ASTFunction>())
{ {
const auto dnf = analyzeFunction(fn, target_expr); const auto dnf = analyzeFunction(fn, target_expr);
@ -350,6 +348,14 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
} }
} }
} }
else if (const auto * literal = node->as<ASTLiteral>())
{
// Check if it's always true or false.
if (literal->value.getType() == Field::Types::UInt64 && literal->value.get<UInt64>() == 0)
return {result};
else
return {};
}
return {result}; return {result};
} }

View File

@ -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()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr)));
volume->getDisk()->moveFile(from, to); volume->getDisk()->moveDirectory(from, to);
relative_path = new_relative_path; relative_path = new_relative_path;
SyncGuardPtr sync_guard; SyncGuardPtr sync_guard;
@ -1065,7 +1065,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
try try
{ {
volume->getDisk()->moveFile(from, to); volume->getDisk()->moveDirectory(from, to);
} }
catch (const Poco::FileNotFoundException &) catch (const Poco::FileNotFoundException &)
{ {

View File

@ -1389,7 +1389,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
DataPartsLock lock(mutex); DataPartsLock lock(mutex);
for (MutableDataPartPtr & part : dst_parts) 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); removePartsFromWorkingSet(src_parts, true, lock);
transaction.commit(&lock); transaction.commit(&lock);

View File

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

View File

@ -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;

View File

@ -0,0 +1 @@
1

View File

@ -0,0 +1 @@
SELECT toUInt8(assumeNotNull(cast(cast(NULL, 'Nullable(String)'), 'Nullable(Enum8(\'Hello\' = 1))')));

View File

@ -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

View File

@ -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;

View File

@ -0,0 +1,10 @@
-9223372036854775807
-9223372036854775808
9223372036854775807
9223372036854775806
9223372036854775805
9223372036854775804
9223372036854775803
9223372036854775802
9223372036854775801
9223372036854775800

View File

@ -0,0 +1 @@
SELECT intDiv(toInt64(number), -1) FROM numbers(9223372036854775807, 10);

View File

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

View File

@ -0,0 +1,2 @@
-- The result is unspecified but UBSan should not argue.
SELECT ignore(addHours(now64(3), inf)) FROM numbers(2);

View File

@ -32,7 +32,10 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS)
add_subdirectory (db-generator) add_subdirectory (db-generator)
add_subdirectory (wal-dump) add_subdirectory (wal-dump)
add_subdirectory (check-mysql-binlog) add_subdirectory (check-mysql-binlog)
add_subdirectory (memcpy-bench)
if (NOT OS_DARWIN)
add_subdirectory (memcpy-bench)
endif ()
endif () endif ()
if (ENABLE_CODE_QUALITY) if (ENABLE_CODE_QUALITY)