Merge branch 'master' into ascii_function

This commit is contained in:
李扬 2022-11-03 22:48:55 -05:00 committed by GitHub
commit fccf6c037f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 202 additions and 100 deletions

View File

@ -3,10 +3,20 @@ option (ENABLE_CLANG_TIDY "Use clang-tidy static analyzer" OFF)
if (ENABLE_CLANG_TIDY) if (ENABLE_CLANG_TIDY)
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12") find_program (CLANG_TIDY_CACHE_PATH NAMES "clang-tidy-cache")
if (CLANG_TIDY_CACHE_PATH)
find_program (_CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12")
# Why do we use ';' here?
# It's a cmake black magic: https://cmake.org/cmake/help/latest/prop_tgt/LANG_CLANG_TIDY.html#prop_tgt:%3CLANG%3E_CLANG_TIDY
# The CLANG_TIDY_PATH is passed to CMAKE_CXX_CLANG_TIDY, which follows CXX_CLANG_TIDY syntax.
set (CLANG_TIDY_PATH "${CLANG_TIDY_CACHE_PATH};${_CLANG_TIDY_PATH}" CACHE STRING "A combined command to run clang-tidy with caching wrapper")
else ()
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12")
endif ()
if (CLANG_TIDY_PATH) if (CLANG_TIDY_PATH)
message(STATUS message (STATUS
"Using clang-tidy: ${CLANG_TIDY_PATH}. "Using clang-tidy: ${CLANG_TIDY_PATH}.
The checks will be run during build process. The checks will be run during build process.
See the .clang-tidy file at the root directory to configure the checks.") See the .clang-tidy file at the root directory to configure the checks.")
@ -15,11 +25,15 @@ if (ENABLE_CLANG_TIDY)
# clang-tidy requires assertions to guide the analysis # clang-tidy requires assertions to guide the analysis
# Note that NDEBUG is set implicitly by CMake for non-debug builds # Note that NDEBUG is set implicitly by CMake for non-debug builds
set(COMPILER_FLAGS "${COMPILER_FLAGS} -UNDEBUG") set (COMPILER_FLAGS "${COMPILER_FLAGS} -UNDEBUG")
# The variable CMAKE_CXX_CLANG_TIDY will be set inside src and base directories with non third-party code. # The variable CMAKE_CXX_CLANG_TIDY will be set inside the following directories with non third-party code.
# - base
# - programs
# - src
# - utils
# set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") # set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
else () else ()
message(${RECONFIGURE_MESSAGE_LEVEL} "clang-tidy is not found") message (${RECONFIGURE_MESSAGE_LEVEL} "clang-tidy is not found")
endif () endif ()
endif () endif ()

View File

@ -91,6 +91,9 @@ ENV PATH="$PATH:/usr/local/go/bin"
ENV GOPATH=/workdir/go ENV GOPATH=/workdir/go
ENV GOCACHE=/workdir/ ENV GOCACHE=/workdir/
RUN curl https://raw.githubusercontent.com/matus-chochlik/ctcache/7fd516e91c17779cbc6fc18bd119313d9532dd90/clang-tidy-cache -Lo /usr/bin/clang-tidy-cache \
&& chmod +x /usr/bin/clang-tidy-cache
RUN mkdir /workdir && chmod 777 /workdir RUN mkdir /workdir && chmod 777 /workdir
WORKDIR /workdir WORKDIR /workdir

View File

@ -258,6 +258,10 @@ def parse_env_variables(
if clang_tidy: if clang_tidy:
# 15G is not enough for tidy build # 15G is not enough for tidy build
cache_maxsize = "25G" cache_maxsize = "25G"
# `CTCACHE_DIR` has the same purpose as the `CCACHE_DIR` above.
# It's there to have the clang-tidy cache embedded into our standard `CCACHE_DIR`
result.append("CTCACHE_DIR=/ccache/clang-tidy-cache")
result.append(f"CCACHE_MAXSIZE={cache_maxsize}") result.append(f"CCACHE_MAXSIZE={cache_maxsize}")
if distcc_hosts: if distcc_hosts:
@ -282,9 +286,7 @@ def parse_env_variables(
cmake_flags.append("-DENABLE_TESTS=1") cmake_flags.append("-DENABLE_TESTS=1")
if shared_libraries: if shared_libraries:
cmake_flags.append( cmake_flags.append("-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1")
"-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1"
)
# We can't always build utils because it requires too much space, but # We can't always build utils because it requires too much space, but
# we have to build them at least in some way in CI. The shared library # we have to build them at least in some way in CI. The shared library
# build is probably the least heavy disk-wise. # build is probably the least heavy disk-wise.

View File

@ -571,13 +571,13 @@ Similar to base58Decode, but returns an empty string in case of error.
## base64Encode(s) ## base64Encode(s)
Encodes s string into base64 Encodes s FixedString or String into base64.
Alias: `TO_BASE64`. Alias: `TO_BASE64`.
## base64Decode(s) ## base64Decode(s)
Decode base64-encoded string s into original string. In case of failure raises an exception. Decode base64-encoded FixedString or String s into original string. In case of failure raises an exception.
Alias: `FROM_BASE64`. Alias: `FROM_BASE64`.

View File

@ -2,21 +2,19 @@
#include "config.h" #include "config.h"
#if USE_BASE64 #if USE_BASE64
# include <Columns/ColumnConst.h> # include <Columns/ColumnFixedString.h>
# include <Common/MemorySanitizer.h>
# include <Columns/ColumnString.h> # include <Columns/ColumnString.h>
# include <DataTypes/DataTypeString.h> # include <DataTypes/DataTypeString.h>
# include <Functions/FunctionFactory.h>
# include <Functions/FunctionHelpers.h> # include <Functions/FunctionHelpers.h>
# include <Functions/GatherUtils/Algorithms.h> # include <Functions/IFunction.h>
# include <IO/WriteHelpers.h> # include <Interpreters/Context_fwd.h>
# include <turbob64.h> # include <turbob64.h>
# include <Common/MemorySanitizer.h>
# include <span>
namespace DB namespace DB
{ {
using namespace GatherUtils;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
@ -25,33 +23,86 @@ namespace ErrorCodes
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
} }
namespace Detail
{
inline size_t base64Decode(const std::span<const UInt8> src, UInt8 * dst)
{
# if defined(__aarch64__)
return tb64sdec(reinterpret_cast<const uint8_t *>(src.data()), src.size(), reinterpret_cast<uint8_t *>(dst));
# else
return _tb64d(reinterpret_cast<const uint8_t *>(src.data()), src.size(), reinterpret_cast<uint8_t *>(dst));
# endif
}
}
struct Base64Encode struct Base64Encode
{ {
static constexpr auto name = "base64Encode"; static constexpr auto name = "base64Encode";
static size_t getBufferSize(size_t string_length, size_t string_count)
static size_t getBufferSize(const size_t string_length, const size_t string_count)
{ {
return ((string_length - string_count) / 3 + string_count) * 4 + 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)
{
/*
* Some bug in sse arm64 implementation?
* `base64Encode(repeat('a', 46))` returns wrong padding character
*/
# if defined(__aarch64__)
return tb64senc(reinterpret_cast<const uint8_t *>(src.data()), src.size(), reinterpret_cast<uint8_t *>(dst));
# else
return _tb64e(reinterpret_cast<const uint8_t *>(src.data()), src.size(), reinterpret_cast<uint8_t *>(dst));
# endif
}
}; };
struct Base64Decode struct Base64Decode
{ {
static constexpr auto name = "base64Decode"; static constexpr auto name = "base64Decode";
static size_t getBufferSize(size_t string_length, size_t string_count) static size_t getBufferSize(const size_t string_length, const size_t string_count)
{ {
return ((string_length - string_count) / 4 + string_count) * 3 + 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)
{
const auto outlen = Detail::base64Decode(src, dst);
if (src.size() > 0 && !outlen)
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Failed to {} input '{}'",
name,
String(reinterpret_cast<const char *>(src.data()), src.size()));
return outlen;
}
}; };
struct TryBase64Decode struct TryBase64Decode
{ {
static constexpr auto name = "tryBase64Decode"; static constexpr auto name = "tryBase64Decode";
static size_t getBufferSize(size_t string_length, size_t string_count) static size_t getBufferSize(const size_t string_length, const size_t string_count)
{ {
return Base64Decode::getBufferSize(string_length, string_count); return Base64Decode::getBufferSize(string_length, string_count);
} }
static size_t performCoding(const std::span<const UInt8> src, UInt8 * dst)
{
if (src.empty())
return 0;
const auto outlen = Detail::base64Decode(src, dst);
// during decoding character array can be partially polluted
// if fail, revert back and clean
if (!outlen)
*dst = 0;
return outlen;
}
}; };
template <typename Func> template <typename Func>
@ -71,99 +122,60 @@ public:
if (arguments.size() != 1) if (arguments.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong number of arguments for function {}: 1 expected.", getName()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong number of arguments for function {}: 1 expected.", getName());
if (!WhichDataType(arguments[0].type).isString()) if (!WhichDataType(arguments[0].type).isStringOrFixedString())
throw Exception( throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}. Must be String.", "Illegal type {} of 1st argument of function {}. Must be FixedString or String.",
arguments[0].type->getName(), getName()); arguments[0].type->getName(),
getName());
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, const size_t input_rows_count) const override
{ {
const ColumnPtr column_string = arguments[0].column; const auto & input_column = arguments[0].column;
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get()); if (const auto * src_column_as_fixed_string = checkAndGetColumn<ColumnFixedString>(*input_column))
return execute(*src_column_as_fixed_string, input_rows_count);
else if (const auto * src_column_as_string = checkAndGetColumn<ColumnString>(*input_column))
return execute(*src_column_as_string, input_rows_count);
if (!input) throw Exception(
throw Exception( ErrorCodes::ILLEGAL_COLUMN,
ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}, must be of type FixedString or String.",
"Illegal column {} of first argument of function {}, must be of type String", input_column->getName(),
arguments[0].column->getName(), getName()); getName());
}
private:
static ColumnPtr execute(const ColumnString & src_column, const size_t src_row_count)
{
auto dst_column = ColumnString::create(); auto dst_column = ColumnString::create();
auto & dst_data = dst_column->getChars(); auto & dst_chars = dst_column->getChars();
auto & dst_offsets = dst_column->getOffsets(); auto & dst_offsets = dst_column->getOffsets();
size_t reserve = Func::getBufferSize(input->getChars().size(), input->size()); const auto reserve = Func::getBufferSize(src_column.byteSize(), src_column.size());
dst_data.resize(reserve); dst_chars.resize(reserve);
dst_offsets.resize(input_rows_count); dst_offsets.resize(src_row_count);
const ColumnString::Offsets & src_offsets = input->getOffsets(); const auto & src_chars = src_column.getChars();
const auto & src_offsets = src_column.getOffsets();
const auto * source = input->getChars().data(); auto * dst = dst_chars.data();
auto * dst = dst_data.data();
auto * dst_pos = dst; auto * dst_pos = dst;
const auto * src = src_chars.data();
size_t src_offset_prev = 0; size_t src_offset_prev = 0;
for (size_t row = 0; row < src_row_count; ++row)
for (size_t row = 0; row < input_rows_count; ++row)
{ {
size_t srclen = src_offsets[row] - src_offset_prev - 1; const size_t src_length = src_offsets[row] - src_offset_prev - 1;
size_t outlen = 0; const auto outlen = Func::performCoding({src, src_length}, dst_pos);
if constexpr (std::is_same_v<Func, Base64Encode>)
{
/*
* Some bug in sse arm64 implementation?
* `base64Encode(repeat('a', 46))` returns wrong padding character
*/
#if defined(__aarch64__)
outlen = tb64senc(reinterpret_cast<const uint8_t *>(source), srclen, reinterpret_cast<uint8_t *>(dst_pos));
#else
outlen = _tb64e(reinterpret_cast<const uint8_t *>(source), srclen, reinterpret_cast<uint8_t *>(dst_pos));
#endif
}
else if constexpr (std::is_same_v<Func, Base64Decode>)
{
if (srclen > 0)
{
#if defined(__aarch64__)
outlen = tb64sdec(reinterpret_cast<const uint8_t *>(source), srclen, reinterpret_cast<uint8_t *>(dst_pos));
#else
outlen = _tb64d(reinterpret_cast<const uint8_t *>(source), srclen, reinterpret_cast<uint8_t *>(dst_pos));
#endif
if (!outlen)
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Failed to {} input '{}'",
getName(), String(reinterpret_cast<const char *>(source), srclen));
}
}
else
{
if (srclen > 0)
{
// during decoding character array can be partially polluted
// if fail, revert back and clean
auto * savepoint = dst_pos;
outlen = _tb64d(reinterpret_cast<const uint8_t *>(source), srclen, reinterpret_cast<uint8_t *>(dst_pos));
if (!outlen)
{
outlen = 0;
dst_pos = savepoint; //-V1048
// clean the symbol
dst_pos[0] = 0;
}
}
}
/// Base64 library is using AVX-512 with some shuffle operations. /// 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. /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.
__msan_unpoison(dst_pos, outlen); __msan_unpoison(dst_pos, outlen);
source += srclen + 1; src += src_length + 1;
dst_pos += outlen; dst_pos += outlen;
*dst_pos = '\0'; *dst_pos = '\0';
dst_pos += 1; dst_pos += 1;
@ -172,8 +184,44 @@ public:
src_offset_prev = src_offsets[row]; src_offset_prev = src_offsets[row];
} }
dst_data.resize(dst_pos - dst); dst_chars.resize(dst_pos - dst);
return dst_column;
}
static ColumnPtr execute(const ColumnFixedString & src_column, const size_t src_row_count)
{
auto dst_column = ColumnString::create();
auto & dst_chars = dst_column->getChars();
auto & dst_offsets = dst_column->getOffsets();
const auto reserve = Func::getBufferSize(src_column.byteSize(), src_column.size());
dst_chars.resize(reserve);
dst_offsets.resize(src_row_count);
const auto & src_chars = src_column.getChars();
const auto & src_n = src_column.getN();
auto * dst = dst_chars.data();
auto * dst_pos = dst;
const auto * src = src_chars.data();
for (size_t row = 0; row < src_row_count; ++row)
{
const auto outlen = Func::performCoding({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.
__msan_unpoison(dst_pos, outlen);
src += src_n;
dst_pos += outlen;
*dst_pos = '\0';
dst_pos += 1;
dst_offsets[row] = dst_pos - dst;
}
dst_chars.resize(dst_pos - dst);
return dst_column; return dst_column;
} }
}; };

View File

@ -1,8 +1,7 @@
#include <Functions/FunctionBase64Conversion.h> #include <Functions/FunctionBase64Conversion.h>
#if USE_BASE64 #if USE_BASE64
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
namespace DB namespace DB
{ {
@ -15,4 +14,5 @@ REGISTER_FUNCTION(Base64Decode)
factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive);
} }
} }
#endif #endif

View File

@ -1,10 +1,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionBase64Conversion.h> #include <Functions/FunctionBase64Conversion.h>
#include "config.h"
#if USE_BASE64 #if USE_BASE64
# include <DataTypes/DataTypeString.h> #include <Functions/FunctionFactory.h>
namespace DB namespace DB
{ {
@ -17,4 +14,5 @@ REGISTER_FUNCTION(Base64Encode)
factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive);
} }
} }
#endif #endif

View File

@ -1,7 +1,7 @@
#include <Functions/FunctionBase64Conversion.h> #include <Functions/FunctionBase64Conversion.h>
#if USE_BASE64 #if USE_BASE64
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
namespace DB namespace DB
{ {
@ -10,4 +10,5 @@ REGISTER_FUNCTION(TryBase64Decode)
factory.registerFunction<FunctionBase64Conversion<TryBase64Decode>>(); factory.registerFunction<FunctionBase64Conversion<TryBase64Decode>>();
} }
} }
#endif #endif

View File

@ -658,7 +658,9 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
/// Save non key columns /// Save non key columns
for (auto & column : sample_block_with_columns_to_add) for (auto & column : sample_block_with_columns_to_add)
{ {
if (!saved_block_sample.findByName(column.name)) if (auto * col = saved_block_sample.findByName(column.name))
*col = column;
else
saved_block_sample.insert(column); saved_block_sample.insert(column);
} }
} }

View File

@ -3,6 +3,7 @@
import argparse import argparse
import csv import csv
import itertools import itertools
import logging
import os import os
from github import Github from github import Github
@ -37,6 +38,8 @@ def process_result(file_path):
state, report_url, description = post_commit_status_from_file(file_path) state, report_url, description = post_commit_status_from_file(file_path)
prefix = os.path.basename(os.path.dirname(file_path)) prefix = os.path.basename(os.path.dirname(file_path))
is_ok = state == "success" is_ok = state == "success"
if is_ok and report_url == "null":
return is_ok, None
status = f'OK: Bug reproduced (<a href="{report_url}">Report</a>' status = f'OK: Bug reproduced (<a href="{report_url}">Report</a>'
if not is_ok: if not is_ok:
@ -51,15 +54,23 @@ def process_all_results(file_paths):
for status_path in file_paths: for status_path in file_paths:
is_ok, test_results = process_result(status_path) is_ok, test_results = process_result(status_path)
any_ok = any_ok or is_ok any_ok = any_ok or is_ok
all_results.extend(test_results) if test_results is not None:
all_results.extend(test_results)
return any_ok, all_results return any_ok, all_results
def main(args): def main(args):
logging.basicConfig(level=logging.INFO)
check_name_with_group = "Bugfix validate check" check_name_with_group = "Bugfix validate check"
is_ok, test_results = process_all_results(args.status) is_ok, test_results = process_all_results(args.status)
if not test_results:
logging.info("No results to upload")
return
pr_info = PRInfo() pr_info = PRInfo()
report_url = upload_results( report_url = upload_results(
S3Helper(), S3Helper(),

View File

@ -14,3 +14,5 @@ fooba
foobar foobar
1 1 1 1
Zm9v
foo

View File

@ -14,3 +14,6 @@ SELECT base64Decode(val, 'excess argument') FROM (select arrayJoin(['', 'Zg==',
SELECT tryBase64Decode('Zm9vYmF=Zm9v', 'excess argument'); -- { serverError 42 } SELECT tryBase64Decode('Zm9vYmF=Zm9v', 'excess argument'); -- { serverError 42 }
SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError 117 } SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError 117 }
select base64Encode(toFixedString('foo', 3));
select base64Decode(toFixedString('Zm9v', 4));

View File

@ -0,0 +1,2 @@
4 6
4 4

View File

@ -0,0 +1,16 @@
DROP TABLE IF EXISTS tab1;
DROP TABLE IF EXISTS tab2;
SET allow_suspicious_low_cardinality_types = 1;
CREATE TABLE tab1 (a1 Int32, b1 Int32, val UInt64) ENGINE = MergeTree ORDER BY a1;
CREATE TABLE tab2 (a2 LowCardinality(Int32), b2 Int32) ENGINE = MergeTree ORDER BY a2;
INSERT INTO tab1 SELECT number, number, 1 from numbers(4);
INSERT INTO tab2 SELECT number + 2, number + 2 from numbers(4);
SELECT sum(val), count(val) FROM tab1 FULL OUTER JOIN tab2 ON b1 - 2 = a2 OR a1 = b2 SETTINGS join_use_nulls = 0;
SELECT sum(val), count(val) FROM tab1 FULL OUTER JOIN tab2 ON b1 - 2 = a2 OR a1 = b2 SETTINGS join_use_nulls = 1;
DROP TABLE IF EXISTS tab1;
DROP TABLE IF EXISTS tab2;