未验证 提交 aa54235d 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #4564 from yandex/better-string-comparison

Better string comparison
......@@ -4,6 +4,7 @@
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/memcpySmall.h>
#include <Common/memcmpSmall.h>
#include <DataStreams/ColumnGathererStream.h>
......@@ -106,8 +107,7 @@ struct ColumnFixedString::less
explicit less(const ColumnFixedString & parent_) : parent(parent_) {}
bool operator()(size_t lhs, size_t rhs) const
{
/// TODO: memcmp slows down.
int res = memcmp(&parent.chars[lhs * parent.n], &parent.chars[rhs * parent.n], parent.n);
int res = memcmpSmallAllowOverflow15(parent.chars.data() + lhs * parent.n, parent.chars.data() + rhs * parent.n, parent.n);
return positive ? (res < 0) : (res > 0);
}
};
......
#pragma once
#include <string.h> // memcmp
#include <Common/PODArray.h>
#include <Common/memcmpSmall.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVectorHelper.h>
......@@ -98,7 +97,7 @@ public:
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_);
return memcmp(&chars[p1 * n], &rhs.chars[p2 * n], n);
return memcmpSmallAllowOverflow15(chars.data() + p1 * n, rhs.chars.data() + p2 * n, n);
}
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
......
#include <Core/Defines.h>
#include <Common/Arena.h>
#include <Common/memcmpSmall.h>
#include <Columns/Collator.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsCommon.h>
......@@ -239,15 +240,11 @@ struct ColumnString::less
explicit less(const ColumnString & parent_) : parent(parent_) {}
bool operator()(size_t lhs, size_t rhs) const
{
size_t left_len = parent.sizeAt(lhs);
size_t right_len = parent.sizeAt(rhs);
int res = memcmpSmallAllowOverflow15(
parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1,
parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1);
int res = memcmp(&parent.chars[parent.offsetAt(lhs)], &parent.chars[parent.offsetAt(rhs)], std::min(left_len, right_len));
if (res != 0)
return positive ? (res < 0) : (res > 0);
else
return positive ? (left_len < right_len) : (left_len > right_len);
return positive ? (res < 0) : (res > 0);
}
};
......
......@@ -6,6 +6,7 @@
#include <Common/PODArray.h>
#include <Common/SipHash.h>
#include <Common/memcpySmall.h>
#include <Common/memcmpSmall.h>
class Collator;
......@@ -210,16 +211,7 @@ public:
int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnString & rhs = static_cast<const ColumnString &>(rhs_);
const size_t size = sizeAt(n);
const size_t rhs_size = rhs.sizeAt(m);
int cmp = memcmp(&chars[offsetAt(n)], &rhs.chars[rhs.offsetAt(m)], std::min(size, rhs_size));
if (cmp != 0)
return cmp;
else
return size > rhs_size ? 1 : (size < rhs_size ? -1 : 0);
return memcmpSmallAllowOverflow15(chars.data() + offsetAt(n), sizeAt(n) - 1, rhs.chars.data() + rhs.offsetAt(m), rhs.sizeAt(m) - 1);
}
/// Variant of compareAt for string comparison with respect of collation.
......
#pragma once
#include <cstdint>
#include <algorithm>
#ifdef __SSE2__
#include <emmintrin.h>
namespace detail
{
template <typename T>
inline int cmp(T a, T b)
{
if (a < b)
return -1;
if (a > b)
return 1;
return 0;
}
}
/** All functions works under the following assumptions:
* - it's possible to read up to 15 excessive bytes after end of 'a' and 'b' region;
* - memory regions are relatively small and extra loop unrolling is not worth to do.
*/
/** Variant when memory regions may have different sizes.
*/
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctz(mask);
if (offset >= min_size)
break;
return detail::cmp(a[offset], b[offset]);
}
}
return detail::cmp(a_size, b_size);
}
/** Variant when memory regions have same size.
* TODO Check if the compiler can optimize previous function when the caller pass identical sizes.
*/
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctz(mask);
if (offset >= size)
return 0;
return detail::cmp(a[offset], b[offset]);
}
}
return 0;
}
/** Compare memory regions for equality.
*/
template <typename Char>
inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
if (a_size != b_size)
return false;
for (size_t offset = 0; offset < a_size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctz(mask);
return offset >= a_size;
}
}
return true;
}
/** Variant when the caller know in advance that the size is a multiple of 16.
*/
template <typename Char>
inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
{
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctz(mask);
return detail::cmp(a[offset], b[offset]);
}
}
return 0;
}
/** Variant when the size is 16 exactly.
*/
template <typename Char>
inline int memcmp16(const Char * a, const Char * b)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
mask = ~mask;
if (mask)
{
auto offset = __builtin_ctz(mask);
return detail::cmp(a[offset], b[offset]);
}
return 0;
}
/** Variant when the size is 16 exactly.
*/
inline bool memequal16(const void * a, const void * b)
{
return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
}
/** Compare memory region to zero */
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
const __m128i zero16 = _mm_setzero_si128();
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(zero16,
_mm_loadu_si128(reinterpret_cast<const __m128i *>(reinterpret_cast<const char *>(data) + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctz(mask);
return offset >= size;
}
}
return true;
}
#else
#include <cstring>
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
return memcmp(a, b, std::min(a_size, b_size));
}
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
return memcmp(a, b, size);
}
template <typename Char>
inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
return a_size == b_size && 0 == memcmp(a, b, a_size);
}
template <typename Char>
inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
{
return memcmp(a, b, size);
}
template <typename Char>
inline int memcmp16(const Char * a, const Char * b)
{
return memcmp(a, b, 16);
}
inline bool memequal16(const void * a, const void * b)
{
return 0 == memcmp(a, b, 16);
}
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
const char * pos = reinterpret_cast<const char *>(data);
const char * end = pos + size;
for (; pos < end; ++pos)
if (*pos)
return false;
return true;
}
#endif
#pragma once
#include <string.h>
#include <Core/Defines.h>
#ifdef __SSE2__
#include <emmintrin.h>
......
#pragma once
#include <cstring>
#include <Common/memcmpSmall.h>
#include <Columns/ColumnString.h>
#include <Functions/FunctionFactory.h>
......@@ -38,11 +39,9 @@ struct EmptyImpl
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
{
std::vector<char> empty_chars(n);
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
res[i] = negative ^ (0 == memcmp(&data[i * n], empty_chars.data(), n));
res[i] = negative ^ memoryIsZeroSmallAllowOverflow15(data.data() + i * n, n);
}
static void array(const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)
......
......@@ -6,9 +6,11 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/FieldVisitors.h>
#include <Common/memcmpSmall.h>
namespace DB
......@@ -272,8 +274,7 @@ struct ArrayIndexNumNullImpl
}
};
/// Implementation for arrays of strings when the 2nd function argument
/// is a NULL value.
/// Implementation for arrays of strings when the 2nd function argument is a NULL value.
template <typename IndexConv>
struct ArrayIndexStringNullImpl
{
......@@ -311,12 +312,11 @@ struct ArrayIndexStringImpl
{
static void vector_const(
const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets,
const String & value,
const ColumnString::Chars & value, ColumnString::Offset value_size,
PaddedPODArray<typename IndexConv::ResultType> & result,
const PaddedPODArray<UInt8> * null_map_data)
{
const auto size = offsets.size();
const auto value_size = value.size();
result.resize(size);
ColumnArray::Offset current_offset = 0;
......@@ -331,12 +331,12 @@ struct ArrayIndexStringImpl
? 0
: string_offsets[current_offset + j - 1];
ColumnArray::Offset string_size = string_offsets[current_offset + j] - string_pos;
ColumnArray::Offset string_size = string_offsets[current_offset + j] - string_pos - 1;
if (null_map_data && (*null_map_data)[current_offset + j])
{
}
else if (string_size == value_size + 1 && 0 == memcmp(value.data(), &data[string_pos], value_size))
else if (memequalSmallAllowOverflow15(value.data(), value_size, &data[string_pos], string_size))
{
if (!IndexConv::apply(j, current))
break;
......@@ -381,7 +381,7 @@ struct ArrayIndexStringImpl
if (null_map_item && (*null_map_item)[i])
hit = true;
}
else if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size))
else if (memequalSmallAllowOverflow15(&item_values[value_pos], value_size, &data[string_pos], string_size))
hit = true;
if (hit)
......@@ -708,16 +708,32 @@ private:
const auto item_arg = block.getByPosition(arguments[1]).column.get();
if (item_arg->onlyNull())
{
ArrayIndexStringNullImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), col_res->getData(), null_map_data);
}
else if (const auto item_arg_const = checkAndGetColumnConstStringOrFixedString(item_arg))
ArrayIndexStringImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), item_arg_const->getValue<String>(), col_res->getData(),
null_map_data);
{
const ColumnString * item_const_string = checkAndGetColumn<ColumnString>(&item_arg_const->getDataColumn());
const ColumnFixedString * item_const_fixedstring = checkAndGetColumn<ColumnFixedString>(&item_arg_const->getDataColumn());
if (item_const_string)
ArrayIndexStringImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(),
item_const_string->getChars(), item_const_string->getDataAt(0).size,
col_res->getData(), null_map_data);
else if (item_const_fixedstring)
ArrayIndexStringImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(),
item_const_fixedstring->getChars(), item_const_fixedstring->getN(),
col_res->getData(), null_map_data);
else
throw Exception("Logical error: ColumnConst contains not String nor FixedString column", ErrorCodes::ILLEGAL_COLUMN);
}
else if (const auto item_arg_vector = checkAndGetColumn<ColumnString>(item_arg))
{
ArrayIndexStringImpl<IndexConv>::vector_vector(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(),
col_res->getData(), null_map_data, null_map_item);
}
else
return false;
......
<test>
<name>String sorting</name>
<preconditions>
<table_exists>hits_10m_single</table_exists>
</preconditions>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>50</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<substitutions>
<substitution>
<name>str1</name>
<values>
<value>URL</value>
<value>Referer</value>
<value>Title</value>
<value>SearchPhrase</value>
<value>MobilePhoneModel</value>
<value>PageCharset</value>
</values>
</substitution>
<substitution>
<name>str2</name>
<values>
<value>URL</value>
<value>Referer</value>
<value>Title</value>
<value>SearchPhrase</value>
<value>MobilePhoneModel</value>
<value>PageCharset</value>
</values>
</substitution>
</substitutions>
<query><![CDATA[SELECT {str1} FROM hits_10m_single ORDER BY {str1} LIMIT 10]]></query>
<query><![CDATA[SELECT {str1} FROM hits_10m_single ORDER BY {str1} LIMIT 9000000, 10]]></query>
<query><![CDATA[SELECT {str1}, {str2} FROM hits_10m_single ORDER BY {str1}, {str2} LIMIT 10]]></query>
<query><![CDATA[SELECT {str1}, {str2} FROM hits_10m_single ORDER BY {str1}, {str2} LIMIT 9000000, 10]]></query>
<main_metric>
<min_time/>
</main_metric>
</test>
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
1 0 0 1 1
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 1 0 1 0
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
1 0 0 1 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 0 1 0 1
0 1 0 1 0
0 0 1 0 1
0 1 0 1 0
0 0 1 0 1
0 1 0 1 0
0 0 1 0 1
0 1 0 1 0
0 0 1 0 1
1
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' AS a, prefix || 'y' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'y' AS a, prefix || 'x' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' AS a, prefix || 'x' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' || prefix AS a, prefix || 'y' || prefix AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'y' || prefix AS a, prefix || 'x' || prefix AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' || prefix AS a, prefix || 'x' || prefix AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' || prefix AS a, prefix || 'y' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'y' || prefix AS a, prefix || 'x' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH substring('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', 1, number) AS prefix, prefix || 'x' || prefix AS a, prefix || 'x' AS b SELECT a = b, a < b, a > b, a <= b, a >= b FROM numbers(40);
WITH arrayJoin(['aaa', 'bbb']) AS a, 'aaa\0bbb' AS b SELECT a = b, a < b, a > b, a <= b, a >= b;
WITH arrayJoin(['aaa', 'zzz']) AS a, 'aaa\0bbb' AS b SELECT a = b, a < b, a > b, a <= b, a >= b;
WITH arrayJoin(['aaa', 'bbb']) AS a, materialize('aaa\0bbb') AS b SELECT a = b, a < b, a > b, a <= b, a >= b;
WITH arrayJoin(['aaa', 'zzz']) AS a, materialize('aaa\0bbb') AS b SELECT a = b, a < b, a > b, a <= b, a >= b;
SELECT empty(toFixedString('', 1 + randConstant() % 100));
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册