Merge branch 'master' into hanfei/doc-cgroups_memory_usage_observer_wait_time

This commit is contained in:
Han Fei 2024-06-28 00:10:18 +02:00
commit 451c3ef1eb
17 changed files with 143 additions and 77 deletions

View File

@ -167,10 +167,16 @@ jobs:
steps: steps:
- name: Check out repository code - name: Check out repository code
uses: ClickHouse/checkout@v1 uses: ClickHouse/checkout@v1
- name: Download reports
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(needs.RunConfig.outputs.data) }} --pre --job-name Builds
- name: Builds report - name: Builds report
run: | run: |
cd "$GITHUB_WORKSPACE/tests/ci" cd "$GITHUB_WORKSPACE/tests/ci"
python3 ./build_report_check.py --reports package_release package_aarch64 package_asan package_tsan package_debug binary_darwin binary_darwin_aarch64 python3 ./build_report_check.py --reports package_release package_aarch64 package_asan package_tsan package_debug binary_darwin binary_darwin_aarch64
- name: Set status
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(needs.RunConfig.outputs.data) }} --post --job-name Builds
############################################################################################ ############################################################################################
#################################### INSTALL PACKAGES ###################################### #################################### INSTALL PACKAGES ######################################
############################################################################################ ############################################################################################

29
.github/workflows/create_release.yml vendored Normal file
View File

@ -0,0 +1,29 @@
name: CreateRelease
concurrency:
group: release
'on':
workflow_dispatch:
inputs:
sha:
description: 'The SHA hash of the commit from which to create the release'
required: true
type: string
type:
description: 'The type of release: "new" for a new release or "patch" for a patch release'
required: true
type: choice
options:
- new
- patch
jobs:
Release:
runs-on: [self-hosted, style-checker-aarch64]
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
- name: Print greeting
run: |
python3 ./tests/ci/release.py --commit ${{ inputs.sha }} --type ${{ inputs.type }} --dry-run

View File

@ -184,10 +184,16 @@ jobs:
steps: steps:
- name: Check out repository code - name: Check out repository code
uses: ClickHouse/checkout@v1 uses: ClickHouse/checkout@v1
- name: Download reports
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(needs.RunConfig.outputs.data) }} --pre --job-name Builds
- name: Builds report - name: Builds report
run: | run: |
cd "$GITHUB_WORKSPACE/tests/ci" cd "$GITHUB_WORKSPACE/tests/ci"
python3 ./build_report_check.py --reports package_release package_aarch64 package_asan package_msan package_ubsan package_tsan package_debug binary_darwin binary_darwin_aarch64 python3 ./build_report_check.py --reports package_release package_aarch64 package_asan package_msan package_ubsan package_tsan package_debug binary_darwin binary_darwin_aarch64
- name: Set status
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(needs.RunConfig.outputs.data) }} --post --job-name Builds
MarkReleaseReady: MarkReleaseReady:
if: ${{ !failure() && !cancelled() }} if: ${{ !failure() && !cancelled() }}
needs: needs:

View File

@ -28,7 +28,7 @@ run, for example, the test `01428_hash_set_nan_key`, change to the repository
folder and run the following command: folder and run the following command:
``` ```
PATH=$PATH:<path to clickhouse-client> tests/clickhouse-test 01428_hash_set_nan_key PATH=<path to clickhouse-client>:$PATH tests/clickhouse-test 01428_hash_set_nan_key
``` ```
Test results (`stderr` and `stdout`) are written to files `01428_hash_set_nan_key.[stderr|stdout]` which Test results (`stderr` and `stdout`) are written to files `01428_hash_set_nan_key.[stderr|stdout]` which

View File

@ -1168,14 +1168,14 @@ Result:
└────────────────────────────┘ └────────────────────────────┘
``` ```
## base64UrlEncode ## base64URLEncode
Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5).
**Syntax** **Syntax**
```sql ```sql
base64UrlEncode(url) base64URLEncode(url)
``` ```
**Arguments** **Arguments**
@ -1189,13 +1189,13 @@ base64UrlEncode(url)
**Example** **Example**
``` sql ``` sql
SELECT base64UrlEncode('https://clickhouse.com'); SELECT base64URLEncode('https://clickhouse.com');
``` ```
Result: Result:
```result ```result
┌─base64UrlEncode('https://clickhouse.com')─┐ ┌─base64URLEncode('https://clickhouse.com')─┐
│ aHR0cDovL2NsaWNraG91c2UuY29t │ │ aHR0cDovL2NsaWNraG91c2UuY29t │
└───────────────────────────────────────────┘ └───────────────────────────────────────────┘
``` ```
@ -1234,19 +1234,19 @@ Result:
└──────────────────────────────────┘ └──────────────────────────────────┘
``` ```
## base64UrlDecode ## base64URLDecode
Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). Throws an exception in case of an error. Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). Throws an exception in case of an error.
**Syntax** **Syntax**
```sql ```sql
base64UrlDecode(encodedUrl) base64URLDecode(encodedUrl)
``` ```
**Arguments** **Arguments**
- `encodedUrl` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, an exception is thrown. - `encodedURL` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, an exception is thrown.
**Returned value** **Returned value**
@ -1255,13 +1255,13 @@ base64UrlDecode(encodedUrl)
**Example** **Example**
``` sql ``` sql
SELECT base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t'); SELECT base64URLDecode('aHR0cDovL2NsaWNraG91c2UuY29t');
``` ```
Result: Result:
```result ```result
┌─base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t')─┐ ┌─base64URLDecode('aHR0cDovL2NsaWNraG91c2UuY29t')─┐
│ https://clickhouse.com │ │ https://clickhouse.com │
└─────────────────────────────────────────────────┘ └─────────────────────────────────────────────────┘
``` ```
@ -1298,19 +1298,19 @@ SELECT tryBase64Decode('RW5jb2RlZA==') as res, tryBase64Decode('invalid') as res
└────────────┴─────────────┘ └────────────┴─────────────┘
``` ```
## tryBase64UrlDecode ## tryBase64URLDecode
Like `base64UrlDecode` but returns an empty string in case of error. Like `base64URLDecode` but returns an empty string in case of error.
**Syntax** **Syntax**
```sql ```sql
tryBase64UrlDecode(encodedUrl) tryBase64URLDecode(encodedUrl)
``` ```
**Parameters** **Parameters**
- `encodedUrl`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. - `encodedURL`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string.
**Returned value** **Returned value**
@ -1321,7 +1321,7 @@ tryBase64UrlDecode(encodedUrl)
Query: Query:
```sql ```sql
SELECT tryBase64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t') as res, tryBase64Decode('aHR0cHM6Ly9jbGlja') as res_invalid; SELECT tryBase64URLDecode('aHR0cDovL2NsaWNraG91c2UuY29t') as res, tryBase64Decode('aHR0cHM6Ly9jbGlja') as res_invalid;
``` ```
```response ```response

View File

@ -134,8 +134,8 @@ PRIMARY KEY (event_type, ts)
ORDER BY (event_type, ts, browser) ORDER BY (event_type, ts, browser)
SETTINGS index_granularity = 8192 SETTINGS index_granularity = 8192
-- !!! The columns' definition is unchanged but it does not matter, we are not quering -- !!! The columns' definition is unchanged but it does not matter, we are not querying
-- MATERIALIZED VIEW, we are quering TO (storage) table. -- MATERIALIZED VIEW, we are querying TO (storage) table.
-- SELECT section is updated. -- SELECT section is updated.
SHOW CREATE TABLE mv FORMAT TSVRaw; SHOW CREATE TABLE mv FORMAT TSVRaw;

View File

@ -538,7 +538,7 @@ SELECT base58Decode('3dc8KtHrwM');
Синоним: `TO_BASE64`. Синоним: `TO_BASE64`.
## base64UrlEncode(s) ## base64URLEncode(s)
Производит кодирование URL (String или FixedString) в base64-представление в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). Производит кодирование URL (String или FixedString) в base64-представление в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648).
@ -548,7 +548,7 @@ SELECT base58Decode('3dc8KtHrwM');
Синоним: `FROM_BASE64`. Синоним: `FROM_BASE64`.
## base64UrlDecode(s) ## base64URLDecode(s)
Декодирует base64-представление URL в исходную строку в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). При невозможности декодирования выбрасывает исключение Декодирует base64-представление URL в исходную строку в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). При невозможности декодирования выбрасывает исключение
@ -556,9 +556,9 @@ SELECT base58Decode('3dc8KtHrwM');
Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку.
## tryBase64UrlDecode(s) ## tryBase64URLDecode(s)
Функционал аналогичен base64UrlDecode, но при невозможности декодирования возвращает пустую строку. Функционал аналогичен base64URLDecode, но при невозможности декодирования возвращает пустую строку.
## endsWith(s, suffix) {#endswith} ## endsWith(s, suffix) {#endswith}

View File

@ -25,10 +25,10 @@ namespace ErrorCodes
enum class Base64Variant : uint8_t enum class Base64Variant : uint8_t
{ {
Normal, Normal,
Url URL
}; };
inline std::string preprocessBase64Url(std::string_view src) inline std::string preprocessBase64URL(std::string_view src)
{ {
std::string padded_src; std::string padded_src;
padded_src.reserve(src.size() + 3); padded_src.reserve(src.size() + 3);
@ -70,7 +70,7 @@ inline std::string preprocessBase64Url(std::string_view src)
return padded_src; return padded_src;
} }
inline size_t postprocessBase64Url(UInt8 * dst, size_t out_len) inline size_t postprocessBase64URL(UInt8 * dst, size_t out_len)
{ {
// Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#section-5 // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#section-5
for (size_t i = 0; i < out_len; ++i) for (size_t i = 0; i < out_len; ++i)
@ -95,7 +95,7 @@ inline size_t postprocessBase64Url(UInt8 * dst, size_t out_len)
template <Base64Variant variant> template <Base64Variant variant>
struct Base64Encode struct Base64Encode
{ {
static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Encode" : "base64UrlEncode"; static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Encode" : "base64URLEncode";
static size_t getBufferSize(size_t string_length, size_t string_count) static size_t getBufferSize(size_t string_length, size_t string_count)
{ {
@ -111,8 +111,8 @@ struct Base64Encode
/// Memory sanitizer doesn't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. /// Memory sanitizer doesn't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle.
__msan_unpoison(dst, outlen); __msan_unpoison(dst, outlen);
if constexpr (variant == Base64Variant::Url) if constexpr (variant == Base64Variant::URL)
outlen = postprocessBase64Url(dst, outlen); outlen = postprocessBase64URL(dst, outlen);
return outlen; return outlen;
} }
@ -121,7 +121,7 @@ struct Base64Encode
template <Base64Variant variant> template <Base64Variant variant>
struct Base64Decode struct Base64Decode
{ {
static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Decode" : "base64UrlDecode"; static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Decode" : "base64URLDecode";
static size_t getBufferSize(size_t string_length, size_t string_count) static size_t getBufferSize(size_t string_length, size_t string_count)
{ {
@ -132,9 +132,9 @@ struct Base64Decode
{ {
int rc; int rc;
size_t outlen = 0; size_t outlen = 0;
if constexpr (variant == Base64Variant::Url) if constexpr (variant == Base64Variant::URL)
{ {
std::string src_padded = preprocessBase64Url(src); std::string src_padded = preprocessBase64URL(src);
rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast<char *>(dst), &outlen, 0); rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast<char *>(dst), &outlen, 0);
} }
else else
@ -156,7 +156,7 @@ struct Base64Decode
template <Base64Variant variant> template <Base64Variant variant>
struct TryBase64Decode struct TryBase64Decode
{ {
static constexpr auto name = (variant == Base64Variant::Normal) ? "tryBase64Decode" : "tryBase64UrlDecode"; static constexpr auto name = (variant == Base64Variant::Normal) ? "tryBase64Decode" : "tryBase64URLDecode";
static size_t getBufferSize(size_t string_length, size_t string_count) static size_t getBufferSize(size_t string_length, size_t string_count)
{ {
@ -167,9 +167,9 @@ struct TryBase64Decode
{ {
int rc; int rc;
size_t outlen = 0; size_t outlen = 0;
if constexpr (variant == Base64Variant::Url) if constexpr (variant == Base64Variant::URL)
{ {
std::string src_padded = preprocessBase64Url(src); std::string src_padded = preprocessBase64URL(src);
rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast<char *>(dst), &outlen, 0); rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast<char *>(dst), &outlen, 0);
} }
else else

View File

@ -5,16 +5,16 @@
namespace DB namespace DB
{ {
REGISTER_FUNCTION(Base64UrlDecode) REGISTER_FUNCTION(Base64URLDecode)
{ {
FunctionDocumentation::Description description = R"(Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)"; FunctionDocumentation::Description description = R"(Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)";
FunctionDocumentation::Syntax syntax = "base64UrlDecode(encodedUrl)"; FunctionDocumentation::Syntax syntax = "base64URLDecode(encodedURL)";
FunctionDocumentation::Arguments arguments = {{"encodedUrl", "String column or constant. If the string is not a valid Base64-encoded value, an exception is thrown."}}; FunctionDocumentation::Arguments arguments = {{"encodedURL", "String column or constant. If the string is not a valid Base64-encoded value, an exception is thrown."}};
FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument.";
FunctionDocumentation::Examples examples = {{"Example", "SELECT base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t')", "https://clickhouse.com"}}; FunctionDocumentation::Examples examples = {{"Example", "SELECT base64URLDecode('aHR0cDovL2NsaWNraG91c2UuY29t')", "https://clickhouse.com"}};
FunctionDocumentation::Categories categories = {"String encoding"}; FunctionDocumentation::Categories categories = {"String encoding"};
factory.registerFunction<FunctionBase64Conversion<Base64Decode<Base64Variant::Url>>>({description, syntax, arguments, returned_value, examples, categories}); factory.registerFunction<FunctionBase64Conversion<Base64Decode<Base64Variant::URL>>>({description, syntax, arguments, returned_value, examples, categories});
} }
} }

View File

@ -5,16 +5,16 @@
namespace DB namespace DB
{ {
REGISTER_FUNCTION(Base64UrlEncode) REGISTER_FUNCTION(Base64URLEncode)
{ {
FunctionDocumentation::Description description = R"(Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)"; FunctionDocumentation::Description description = R"(Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)";
FunctionDocumentation::Syntax syntax = "base64UrlEncode(url)"; FunctionDocumentation::Syntax syntax = "base64URLEncode(url)";
FunctionDocumentation::Arguments arguments = {{"url", "String column or constant."}}; FunctionDocumentation::Arguments arguments = {{"url", "String column or constant."}};
FunctionDocumentation::ReturnedValue returned_value = "A string containing the encoded value of the argument."; FunctionDocumentation::ReturnedValue returned_value = "A string containing the encoded value of the argument.";
FunctionDocumentation::Examples examples = {{"Example", "SELECT base64UrlEncode('https://clickhouse.com')", "aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ"}}; FunctionDocumentation::Examples examples = {{"Example", "SELECT base64URLEncode('https://clickhouse.com')", "aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ"}};
FunctionDocumentation::Categories categories = {"String encoding"}; FunctionDocumentation::Categories categories = {"String encoding"};
factory.registerFunction<FunctionBase64Conversion<Base64Encode<Base64Variant::Url>>>({description, syntax, arguments, returned_value, examples, categories}); factory.registerFunction<FunctionBase64Conversion<Base64Encode<Base64Variant::URL>>>({description, syntax, arguments, returned_value, examples, categories});
} }
} }

View File

@ -5,16 +5,16 @@
namespace DB namespace DB
{ {
REGISTER_FUNCTION(TryBase64UrlDecode) REGISTER_FUNCTION(TryBase64URLDecode)
{ {
FunctionDocumentation::Description description = R"(Decodes an URL from base64, like base64UrlDecode but returns an empty string in case of an error.)"; FunctionDocumentation::Description description = R"(Decodes an URL from base64, like base64URLDecode but returns an empty string in case of an error.)";
FunctionDocumentation::Syntax syntax = "tryBase64UrlDecode(encodedUrl)"; FunctionDocumentation::Syntax syntax = "tryBase64URLDecode(encodedUrl)";
FunctionDocumentation::Arguments arguments = {{"encodedUrl", "String column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string."}}; FunctionDocumentation::Arguments arguments = {{"encodedURL", "String column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string."}};
FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument.";
FunctionDocumentation::Examples examples = {{"valid", "SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ')", "https://clickhouse.com"}, {"invalid", "SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja')", ""}}; FunctionDocumentation::Examples examples = {{"valid", "SELECT tryBase64URLDecode('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ')", "https://clickhouse.com"}, {"invalid", "SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja')", ""}};
FunctionDocumentation::Categories categories = {"String encoding"}; FunctionDocumentation::Categories categories = {"String encoding"};
factory.registerFunction<FunctionBase64Conversion<TryBase64Decode<Base64Variant::Url>>>({description, syntax, arguments, returned_value, examples, categories}); factory.registerFunction<FunctionBase64Conversion<TryBase64Decode<Base64Variant::URL>>>({description, syntax, arguments, returned_value, examples, categories});
} }
} }

View File

@ -9,7 +9,6 @@
#include <base/defines.h> #include <base/defines.h>
#include <base/types.h> #include <base/types.h>
#include <Common/logger_useful.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
@ -19,6 +18,7 @@
#include <Interpreters/FullSortingMergeJoin.h> #include <Interpreters/FullSortingMergeJoin.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Processors/Chunk.h>
#include <Processors/Transforms/MergeJoinTransform.h> #include <Processors/Transforms/MergeJoinTransform.h>
@ -40,7 +40,7 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns)
desc.reserve(columns.size()); desc.reserve(columns.size());
for (const auto & name : columns) for (const auto & name : columns)
desc.emplace_back(name); desc.emplace_back(name);
return std::make_unique<FullMergeJoinCursor>(materializeBlock(block), desc); return std::make_unique<FullMergeJoinCursor>(block, desc);
} }
template <bool has_left_nulls, bool has_right_nulls> template <bool has_left_nulls, bool has_right_nulls>
@ -234,9 +234,14 @@ void inline addMany(PaddedPODArray<UInt64> & left_or_right_map, size_t idx, size
for (size_t i = 0; i < num; ++i) for (size_t i = 0; i < num; ++i)
left_or_right_map.push_back(idx); left_or_right_map.push_back(idx);
} }
} }
FullMergeJoinCursor::FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_)
: sample_block(materializeBlock(sample_block_).cloneEmpty()), desc(description_)
{
}
const Chunk & FullMergeJoinCursor::getCurrent() const const Chunk & FullMergeJoinCursor::getCurrent() const
{ {
return current_chunk; return current_chunk;
@ -260,6 +265,10 @@ void FullMergeJoinCursor::setChunk(Chunk && chunk)
return; return;
} }
// should match the structure of sample_block (after materialization)
convertToFullIfConst(chunk);
convertToFullIfSparse(chunk);
current_chunk = std::move(chunk); current_chunk = std::move(chunk);
cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), desc); cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), desc);
} }

View File

@ -193,11 +193,7 @@ private:
class FullMergeJoinCursor : boost::noncopyable class FullMergeJoinCursor : boost::noncopyable
{ {
public: public:
explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_);
: sample_block(sample_block_.cloneEmpty())
, desc(description_)
{
}
bool fullyCompleted() const; bool fullyCompleted() const;
void setChunk(Chunk && chunk); void setChunk(Chunk && chunk);

View File

@ -2,35 +2,35 @@
-- no-fasttest because aklomp-base64 library is required -- no-fasttest because aklomp-base64 library is required
-- incorrect number of arguments -- incorrect number of arguments
SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64URLEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT base64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64URLDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT tryBase64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT tryBase64URLDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT base64UrlEncode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64URLEncode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT base64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64URLDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT tryBase64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT tryBase64URLDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
-- test with valid inputs -- test with valid inputs
SELECT 'https://clickhouse.com' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT 'https://clickhouse.com' AS original, base64URLEncode(original) AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
SELECT '12?' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT '12?' AS original, base64URLEncode(original) AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' AS original, base64URLEncode(original) AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
-- encoded value has no padding -- encoded value has no padding
SELECT 'aHR0cHM6Ly9jbGlj' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT 'aHR0cHM6Ly9jbGlj' AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
-- encoded value has one-byte padding -- encoded value has one-byte padding
SELECT 'aHR0cHM6Ly9jbGlja2g' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT 'aHR0cHM6Ly9jbGlja2g' AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
-- encoded value has two-bytes padding -- encoded value has two-bytes padding
SELECT 'aHR0cHM6Ly9jbGljaw' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT 'aHR0cHM6Ly9jbGljaw' AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);
-- test with invalid inputs -- test with invalid inputs
SELECT base64UrlDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA } SELECT base64URLDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA }
SELECT tryBase64UrlDecode('https://clickhouse.com'); SELECT tryBase64URLDecode('https://clickhouse.com');
SELECT base64UrlDecode('12?'); -- { serverError INCORRECT_DATA } SELECT base64URLDecode('12?'); -- { serverError INCORRECT_DATA }
SELECT tryBase64UrlDecode('12?'); SELECT tryBase64URLDecode('12?');
SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja'); -- { serverError INCORRECT_DATA } SELECT base64URLDecode('aHR0cHM6Ly9jbGlja'); -- { serverError INCORRECT_DATA }
SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja'); SELECT tryBase64URLDecode('aHR0cHM6Ly9jbGlja');
-- test FixedString argument -- test FixedString argument
SELECT toFixedString('https://clickhouse.com', 22) AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); SELECT toFixedString('https://clickhouse.com', 22) AS original, base64URLEncode(original) AS encoded, base64URLDecode(encoded), tryBase64URLDecode(encoded);

View File

@ -0,0 +1,10 @@
0 0
1 1
2 2
3 3
4 4
5 5
6 6
7 7
8 8
9 9

View File

@ -0,0 +1,10 @@
-- Tags: no-random-settings
SET allow_experimental_analyzer=1, join_algorithm = 'full_sorting_merge';
CREATE TABLE xxxx_yyy (key UInt32, key_b ALIAS key) ENGINE=MergeTree() ORDER BY key SETTINGS ratio_of_defaults_for_sparse_serialization=0.0;
INSERT INTO xxxx_yyy SELECT number FROM numbers(10);
SELECT *
FROM xxxx_yyy AS a
INNER JOIN xxxx_yyy AS b ON a.key = b.key_b
ORDER BY a.key;

View File

@ -1014,8 +1014,8 @@ Updatable
Uppercased Uppercased
Uptime Uptime
Uptrace Uptrace
UrlDecode URLDecode
UrlEncode URLEncode
UserID UserID
Util Util
VARCHAR VARCHAR