mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into ascii_function
This commit is contained in:
commit
fccf6c037f
@ -3,10 +3,20 @@ option (ENABLE_CLANG_TIDY "Use clang-tidy static analyzer" OFF)
|
||||
|
||||
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)
|
||||
message(STATUS
|
||||
message (STATUS
|
||||
"Using clang-tidy: ${CLANG_TIDY_PATH}.
|
||||
The checks will be run during build process.
|
||||
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
|
||||
# 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}")
|
||||
else ()
|
||||
message(${RECONFIGURE_MESSAGE_LEVEL} "clang-tidy is not found")
|
||||
message (${RECONFIGURE_MESSAGE_LEVEL} "clang-tidy is not found")
|
||||
endif ()
|
||||
endif ()
|
||||
|
@ -91,6 +91,9 @@ ENV PATH="$PATH:/usr/local/go/bin"
|
||||
ENV GOPATH=/workdir/go
|
||||
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
|
||||
WORKDIR /workdir
|
||||
|
||||
|
@ -258,6 +258,10 @@ def parse_env_variables(
|
||||
if clang_tidy:
|
||||
# 15G is not enough for tidy build
|
||||
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}")
|
||||
|
||||
if distcc_hosts:
|
||||
@ -282,9 +286,7 @@ def parse_env_variables(
|
||||
cmake_flags.append("-DENABLE_TESTS=1")
|
||||
|
||||
if shared_libraries:
|
||||
cmake_flags.append(
|
||||
"-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1"
|
||||
)
|
||||
cmake_flags.append("-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1")
|
||||
# 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
|
||||
# build is probably the least heavy disk-wise.
|
||||
|
@ -571,13 +571,13 @@ Similar to base58Decode, but returns an empty string in case of error.
|
||||
|
||||
## base64Encode(s)
|
||||
|
||||
Encodes ‘s’ string into base64
|
||||
Encodes ‘s’ FixedString or String into base64.
|
||||
|
||||
Alias: `TO_BASE64`.
|
||||
|
||||
## 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`.
|
||||
|
||||
|
@ -2,21 +2,19 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_BASE64
|
||||
# include <Columns/ColumnConst.h>
|
||||
# include <Common/MemorySanitizer.h>
|
||||
# include <Columns/ColumnFixedString.h>
|
||||
# include <Columns/ColumnString.h>
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/FunctionHelpers.h>
|
||||
# include <Functions/GatherUtils/Algorithms.h>
|
||||
# include <IO/WriteHelpers.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <turbob64.h>
|
||||
# include <Common/MemorySanitizer.h>
|
||||
|
||||
# include <span>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using namespace GatherUtils;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -25,33 +23,86 @@ namespace ErrorCodes
|
||||
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
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
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>
|
||||
@ -71,99 +122,60 @@ public:
|
||||
if (arguments.size() != 1)
|
||||
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(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 1st argument of function {}. Must be String.",
|
||||
arguments[0].type->getName(), getName());
|
||||
"Illegal type {} of 1st argument of function {}. Must be FixedString or String.",
|
||||
arguments[0].type->getName(),
|
||||
getName());
|
||||
|
||||
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 ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
|
||||
const auto & input_column = arguments[0].column;
|
||||
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(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}, must be of type String",
|
||||
arguments[0].column->getName(), getName());
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}, must be of type FixedString or String.",
|
||||
input_column->getName(),
|
||||
getName());
|
||||
}
|
||||
|
||||
private:
|
||||
static ColumnPtr execute(const ColumnString & src_column, const size_t src_row_count)
|
||||
{
|
||||
auto dst_column = ColumnString::create();
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_chars = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
size_t reserve = Func::getBufferSize(input->getChars().size(), input->size());
|
||||
dst_data.resize(reserve);
|
||||
dst_offsets.resize(input_rows_count);
|
||||
const auto reserve = Func::getBufferSize(src_column.byteSize(), src_column.size());
|
||||
dst_chars.resize(reserve);
|
||||
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_data.data();
|
||||
auto * dst = dst_chars.data();
|
||||
auto * dst_pos = dst;
|
||||
const auto * src = src_chars.data();
|
||||
|
||||
size_t src_offset_prev = 0;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
for (size_t row = 0; row < src_row_count; ++row)
|
||||
{
|
||||
size_t srclen = src_offsets[row] - src_offset_prev - 1;
|
||||
size_t outlen = 0;
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
const size_t src_length = src_offsets[row] - src_offset_prev - 1;
|
||||
const auto outlen = Func::performCoding({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.
|
||||
__msan_unpoison(dst_pos, outlen);
|
||||
|
||||
source += srclen + 1;
|
||||
src += src_length + 1;
|
||||
dst_pos += outlen;
|
||||
*dst_pos = '\0';
|
||||
dst_pos += 1;
|
||||
@ -172,8 +184,44 @@ public:
|
||||
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;
|
||||
}
|
||||
};
|
||||
|
@ -1,8 +1,7 @@
|
||||
#include <Functions/FunctionBase64Conversion.h>
|
||||
|
||||
#if USE_BASE64
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -15,4 +14,5 @@ REGISTER_FUNCTION(Base64Decode)
|
||||
factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,10 +1,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionBase64Conversion.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_BASE64
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -17,4 +14,5 @@ REGISTER_FUNCTION(Base64Encode)
|
||||
factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Functions/FunctionBase64Conversion.h>
|
||||
|
||||
#if USE_BASE64
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -10,4 +10,5 @@ REGISTER_FUNCTION(TryBase64Decode)
|
||||
factory.registerFunction<FunctionBase64Conversion<TryBase64Decode>>();
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -658,7 +658,9 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
|
||||
/// Save non key columns
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
import argparse
|
||||
import csv
|
||||
import itertools
|
||||
import logging
|
||||
import os
|
||||
|
||||
from github import Github
|
||||
@ -37,6 +38,8 @@ def process_result(file_path):
|
||||
state, report_url, description = post_commit_status_from_file(file_path)
|
||||
prefix = os.path.basename(os.path.dirname(file_path))
|
||||
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>'
|
||||
if not is_ok:
|
||||
@ -51,15 +54,23 @@ def process_all_results(file_paths):
|
||||
for status_path in file_paths:
|
||||
is_ok, test_results = process_result(status_path)
|
||||
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
|
||||
|
||||
|
||||
def main(args):
|
||||
logging.basicConfig(level=logging.INFO)
|
||||
|
||||
check_name_with_group = "Bugfix validate check"
|
||||
|
||||
is_ok, test_results = process_all_results(args.status)
|
||||
|
||||
if not test_results:
|
||||
logging.info("No results to upload")
|
||||
return
|
||||
|
||||
pr_info = PRInfo()
|
||||
report_url = upload_results(
|
||||
S3Helper(),
|
||||
|
@ -14,3 +14,5 @@ fooba
|
||||
foobar
|
||||
1 1
|
||||
|
||||
Zm9v
|
||||
foo
|
||||
|
@ -14,3 +14,6 @@ SELECT base64Decode(val, 'excess argument') FROM (select arrayJoin(['', 'Zg==',
|
||||
SELECT tryBase64Decode('Zm9vYmF=Zm9v', 'excess argument'); -- { serverError 42 }
|
||||
|
||||
SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError 117 }
|
||||
|
||||
select base64Encode(toFixedString('foo', 3));
|
||||
select base64Decode(toFixedString('Zm9v', 4));
|
||||
|
2
tests/queries/0_stateless/02475_join_bug_42832.reference
Normal file
2
tests/queries/0_stateless/02475_join_bug_42832.reference
Normal file
@ -0,0 +1,2 @@
|
||||
4 6
|
||||
4 4
|
16
tests/queries/0_stateless/02475_join_bug_42832.sql
Normal file
16
tests/queries/0_stateless/02475_join_bug_42832.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user