Various fixups

This commit is contained in:
Robert Schulze 2023-09-07 09:50:49 +00:00
parent b067213c5a
commit cd3c121a62
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
9 changed files with 54 additions and 101 deletions

View File

@ -1,8 +1,4 @@
if(ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_S390X)
option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES})
elseif(ENABLE_BASE64)
message (${RECONFIGURE_MESSAGE_LEVEL} "base64 library is only supported on x86_64 and aarch64")
endif()
option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES})
if (NOT ENABLE_BASE64)
message(STATUS "Not using base64")
@ -13,7 +9,7 @@ SET(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aklomp-base64")
set(HAVE_NEON32 0)
set(HAVE_NEON64 0)
if (ARCH_AARCH64)
if (ARCH_AARCH64 AND NOT NO_ARMV81_OR_HIGHER)
set(HAVE_NEON64 1)
endif ()
@ -33,7 +29,7 @@ cast_to_bool(AVX "-mavx")
cast_to_bool(AVX2 "-mavx2")
set(HAVE_FAST_UNALIGNED_ACCESS 0)
if(HAVE_SSSE3 OR HAVE_SSE41 OR HAVE_SSE42 OR HAVE_AVX OR HAVE_AVX2 OR ARCH_AARCH64)
if(HAVE_SSSE3 OR HAVE_SSE41 OR HAVE_SSE42 OR HAVE_AVX OR HAVE_AVX2 OR HAVE_NEON64)
set(HAVE_FAST_UNALIGNED_ACCESS 1)
endif ()
@ -48,23 +44,21 @@ add_library(_base64
# Codec implementations
"${LIBRARY_DIR}/lib/arch/generic/codec.c"
"${LIBRARY_DIR}/lib/arch/neon32/codec.c"
"${LIBRARY_DIR}/lib/arch/neon64/codec.c"
"${LIBRARY_DIR}/lib/arch/ssse3/codec.c"
"${LIBRARY_DIR}/lib/arch/sse41/codec.c"
"${LIBRARY_DIR}/lib/arch/sse42/codec.c"
# ClickHouse does not support AVX* by default. We still need to compile these files
# because the run-time codec detection requires functions from these file to be present.
"${LIBRARY_DIR}/lib/arch/avx/codec.c"
"${LIBRARY_DIR}/lib/arch/avx2/codec.c"
# Even though AVX512 is not supported, we still need to compile this file
# because the run-time codec choose requires the functions from this file
# to be present.
"${LIBRARY_DIR}/lib/arch/avx512/codec.c"
# Tables
"${LIBRARY_DIR}/lib/tables/table_dec_32bit.h"
"${LIBRARY_DIR}/lib/tables/table_enc_12bit.h"
"${LIBRARY_DIR}/lib/arch/neon32/codec.c"
"${LIBRARY_DIR}/lib/arch/neon64/codec.c"
"${LIBRARY_DIR}/lib/codecs.h"
"${CMAKE_CURRENT_BINARY_DIR}/config.h")

View File

@ -20,21 +20,10 @@ SELECT library_name, license_type, license_path FROM system.licenses LIMIT 15
``` text
┌─library_name───────┬─license_type─┬─license_path────────────────────────┐
│ FastMemcpy │ MIT │ /contrib/FastMemcpy/LICENSE │
│ arrow │ Apache │ /contrib/arrow/LICENSE.txt │
│ avro │ Apache │ /contrib/avro/LICENSE.txt │
│ aws-c-common │ Apache │ /contrib/aws-c-common/LICENSE │
│ aws-c-event-stream │ Apache │ /contrib/aws-c-event-stream/LICENSE │
│ aws-checksums │ Apache │ /contrib/aws-checksums/LICENSE │
│ aws │ Apache │ /contrib/aws/LICENSE.txt │
│ base64 │ BSD 2-clause │ /contrib/aklomp-base64/LICENSE │
│ boost │ Boost │ /contrib/boost/LICENSE_1_0.txt │
│ brotli │ MIT │ /contrib/brotli/LICENSE │
│ capnproto │ MIT │ /contrib/capnproto/LICENSE │
│ cassandra │ Apache │ /contrib/cassandra/LICENSE.txt │
│ cctz │ Apache │ /contrib/cctz/LICENSE.txt │
│ cityhash102 │ MIT │ /contrib/cityhash102/COPYING │
│ cppkafka │ BSD 2-clause │ /contrib/cppkafka/LICENSE │
│ [...] │ [...] │ [...] │
└────────────────────┴──────────────┴─────────────────────────────────────┘
```

View File

@ -20,21 +20,10 @@ SELECT library_name, license_type, license_path FROM system.licenses LIMIT 15
``` text
┌─library_name───────┬─license_type─┬─license_path────────────────────────┐
│ FastMemcpy │ MIT │ /contrib/FastMemcpy/LICENSE │
│ arrow │ Apache │ /contrib/arrow/LICENSE.txt │
│ avro │ Apache │ /contrib/avro/LICENSE.txt │
│ aws-c-common │ Apache │ /contrib/aws-c-common/LICENSE │
│ aws-c-event-stream │ Apache │ /contrib/aws-c-event-stream/LICENSE │
│ aws-checksums │ Apache │ /contrib/aws-checksums/LICENSE │
│ aws │ Apache │ /contrib/aws/LICENSE.txt │
│ base64 │ BSD 2-clause │ /contrib/aklomp-base64/LICENSE │
│ boost │ Boost │ /contrib/boost/LICENSE_1_0.txt │
│ brotli │ MIT │ /contrib/brotli/LICENSE │
│ capnproto │ MIT │ /contrib/capnproto/LICENSE │
│ cassandra │ Apache │ /contrib/cassandra/LICENSE.txt │
│ cctz │ Apache │ /contrib/cctz/LICENSE.txt │
│ cityhash102 │ MIT │ /contrib/cityhash102/COPYING │
│ cppkafka │ BSD 2-clause │ /contrib/cppkafka/LICENSE │
│ [...] │ [...] │ [...] │
└────────────────────┴──────────────┴─────────────────────────────────────┘
```

View File

@ -20,21 +20,9 @@ SELECT library_name, license_type, license_path FROM system.licenses LIMIT 15
``` text
┌─library_name───────┬─license_type─┬─license_path────────────────────────┐
│ FastMemcpy │ MIT │ /contrib/FastMemcpy/LICENSE │
│ arrow │ Apache │ /contrib/arrow/LICENSE.txt │
│ avro │ Apache │ /contrib/avro/LICENSE.txt │
│ aws-c-common │ Apache │ /contrib/aws-c-common/LICENSE │
│ aws-c-event-stream │ Apache │ /contrib/aws-c-event-stream/LICENSE │
│ aws-checksums │ Apache │ /contrib/aws-checksums/LICENSE │
│ aws │ Apache │ /contrib/aws/LICENSE.txt │
│ base64 │ BSD 2-clause │ /contrib/aklomp-base64/LICENSE │
│ boost │ Boost │ /contrib/boost/LICENSE_1_0.txt │
│ brotli │ MIT │ /contrib/brotli/LICENSE │
│ capnproto │ MIT │ /contrib/capnproto/LICENSE │
│ cassandra │ Apache │ /contrib/cassandra/LICENSE.txt │
│ cctz │ Apache │ /contrib/cctz/LICENSE.txt │
│ cityhash102 │ MIT │ /contrib/cityhash102/COPYING │
│ cppkafka │ BSD 2-clause │ /contrib/cppkafka/LICENSE │
│ [...] │ [...] │ [...] │
└────────────────────┴──────────────┴─────────────────────────────────────┘
```

View File

@ -1,5 +1,4 @@
#pragma once
#include <cstddef>
#include "config.h"
#if USE_BASE64
@ -12,6 +11,7 @@
# include <libbase64.h>
# include <Common/MemorySanitizer.h>
# include <cstddef>
# include <span>
namespace DB
@ -24,26 +24,16 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
}
namespace Detail
{
inline size_t base64Decode(const std::span<const UInt8> src, UInt8 * dst)
{
size_t outlen = 0;
base64_decode(reinterpret_cast<const char *>(src.data()), src.size(), reinterpret_cast<char *>(dst), &outlen, 0);
return outlen;
}
}
struct Base64Encode
{
static constexpr auto name = "base64Encode";
static size_t getBufferSize(const size_t string_length, const size_t string_count)
static size_t getBufferSize(size_t string_length, size_t string_count)
{
return ((string_length - string_count) / 3 + string_count) * 4 + string_count;
}
static size_t performCoding(const std::span<const UInt8> src, UInt8 * dst)
static size_t perform(const std::span<const UInt8> src, UInt8 * dst)
{
size_t outlen = 0;
base64_encode(reinterpret_cast<const char *>(src.data()), src.size(), reinterpret_cast<char *>(dst), &outlen, 0);
@ -55,15 +45,17 @@ struct Base64Decode
{
static constexpr auto name = "base64Decode";
static size_t getBufferSize(const size_t string_length, const size_t string_count)
static size_t getBufferSize(size_t string_length, size_t string_count)
{
return ((string_length - string_count) / 4 + string_count) * 3 + string_count;
}
static size_t performCoding(const std::span<const UInt8> src, UInt8 * dst)
static size_t perform(const std::span<const UInt8> src, UInt8 * dst)
{
const auto outlen = Detail::base64Decode(src, dst);
if (src.size() > 0 && !outlen)
size_t outlen = 0;
int rc = base64_decode(reinterpret_cast<const char *>(src.data()), src.size(), reinterpret_cast<char *>(dst), &outlen, 0);
if (rc != 1)
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Failed to {} input '{}'",
@ -78,17 +70,16 @@ struct TryBase64Decode
{
static constexpr auto name = "tryBase64Decode";
static size_t getBufferSize(const size_t string_length, const size_t string_count)
static size_t getBufferSize(size_t string_length, size_t string_count)
{
return Base64Decode::getBufferSize(string_length, string_count);
}
static size_t performCoding(const std::span<const UInt8> src, UInt8 * dst)
static size_t perform(const std::span<const UInt8> src, UInt8 * dst)
{
if (src.empty())
return 0;
size_t outlen = 0;
base64_decode(reinterpret_cast<const char *>(src.data()), src.size(), reinterpret_cast<char *>(dst), &outlen, 0);
const auto outlen = Detail::base64Decode(src, dst);
// during decoding character array can be partially polluted
// if fail, revert back and clean
if (!outlen)
@ -112,20 +103,16 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong number of arguments for function {}: 1 expected.", getName());
FunctionArgumentDescriptors mandatory_arguments{
{"value", &isStringOrFixedString<IDataType>, nullptr, "String or FixedString"}
};
if (!WhichDataType(arguments[0].type).isStringOrFixedString())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}. Must be FixedString or String.",
arguments[0].type->getName(),
getName());
validateFunctionArgumentTypes(*this, arguments, mandatory_arguments);
return std::make_shared<DataTypeString>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, const size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto & input_column = arguments[0].column;
if (const auto * src_column_as_fixed_string = checkAndGetColumn<ColumnFixedString>(*input_column))
@ -141,7 +128,7 @@ public:
}
private:
static ColumnPtr execute(const ColumnString & src_column, const size_t src_row_count)
static ColumnPtr execute(const ColumnString & src_column, size_t src_row_count)
{
auto dst_column = ColumnString::create();
auto & dst_chars = dst_column->getChars();
@ -162,7 +149,7 @@ private:
for (size_t row = 0; row < src_row_count; ++row)
{
const size_t src_length = src_offsets[row] - src_offset_prev - 1;
const auto outlen = Func::performCoding({src, src_length}, dst_pos);
const auto outlen = Func::perform({src, src_length}, dst_pos);
/// Base64 library is using AVX-512 with some shuffle operations.
/// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.
@ -181,7 +168,7 @@ private:
return dst_column;
}
static ColumnPtr execute(const ColumnFixedString & src_column, const size_t src_row_count)
static ColumnPtr execute(const ColumnFixedString & src_column, size_t src_row_count)
{
auto dst_column = ColumnString::create();
auto & dst_chars = dst_column->getChars();
@ -200,7 +187,7 @@ private:
for (size_t row = 0; row < src_row_count; ++row)
{
const auto outlen = Func::performCoding({src, src_n}, dst_pos);
const auto outlen = Func::perform({src, src_n}, dst_pos);
/// Base64 library is using AVX-512 with some shuffle operations.
/// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.

View File

@ -12,10 +12,17 @@ foo
foob
fooba
foobar
1 1
This used to be an error
f
fo
foo
foob
fooba
This also used to be an error
foobar
1 1
1 1
fooba
Zm9v
foo
foo
TEcgT3B0aW11cw==

View File

@ -5,17 +5,21 @@ SET send_logs_level = 'fatal';
SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val);
SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val);
SELECT tryBase64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val);
SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v';
SELECT tryBase64Decode(base64Encode('foo')) = 'foo', base64Encode(tryBase64Decode('Zm9v')) == 'Zm9v';
SELECT 'This used to be an error';
SELECT base64Encode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT base64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT tryBase64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError INCORRECT_DATA }
SELECT tryBase64Decode('Zm9vYmF=Zm9v');
SELECT base64Encode(val, 'excess argument') FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); -- { serverError 42 }
SELECT base64Decode(val, 'excess argument') FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); -- { serverError 42 }
SELECT tryBase64Decode('Zm9vYmF=Zm9v', 'excess argument'); -- { serverError 42 }
SELECT 'This also used to be an error';
SELECT base64Decode('Zm9vYmF=Zm9v');
select base64Encode(toFixedString('foo', 3));
select base64Decode(toFixedString('Zm9v', 4));
select tryBase64Decode(toFixedString('Zm9v', 4));
-- This query reproduces a bug in TurboBase64 library (which we no longer use)
select distinct base64Encode(materialize('LG Optimus')) from numbers(100);

View File

@ -1 +0,0 @@
TEcgT3B0aW11cw==

View File

@ -1,4 +0,0 @@
-- Tags: no-fasttest
-- This query reproduces a bug in TurboBase64 library.
select distinct base64Encode(materialize('LG Optimus')) from numbers(100);