mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge branch 'master' into pytest
This commit is contained in:
commit
679ce12bc9
@ -38,7 +38,7 @@
|
||||
|
||||
/* Implementation: */
|
||||
|
||||
static int _libcpiud_errno = ERR_OK;
|
||||
_Thread_local int _libcpiud_errno = ERR_OK;
|
||||
|
||||
int set_error(cpu_error_t err)
|
||||
{
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Common/config.h>
|
||||
#if USE_CPUID
|
||||
# include <libcpuid/libcpuid.h>
|
||||
# include <Common/Exception.h>
|
||||
#elif USE_CPUINFO
|
||||
# include <cpuinfo.h>
|
||||
#endif
|
||||
|
@ -1,709 +0,0 @@
|
||||
#include "FunctionsStringSearch.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsMultiStringPosition.h>
|
||||
#include <Functions/FunctionsMultiStringSearch.h>
|
||||
#include <Functions/Regexps.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/UTF8String.h>
|
||||
#include <Common/Volnitsky.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
|
||||
*/
|
||||
struct PositionCaseSensitiveASCII
|
||||
{
|
||||
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
|
||||
using SearcherInBigHaystack = Volnitsky;
|
||||
|
||||
/// For search many substrings in one string
|
||||
using MultiSearcherInBigHaystack = MultiVolnitsky;
|
||||
|
||||
/// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization.
|
||||
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
||||
|
||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||
{
|
||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||
}
|
||||
|
||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||
{
|
||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||
}
|
||||
|
||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||
{
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
|
||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||
|
||||
/// Convert string to lowercase. Only for case-insensitive search.
|
||||
/// Implementation is permitted to be inefficient because it is called for single string.
|
||||
static void toLowerIfNeed(std::string &) { }
|
||||
};
|
||||
|
||||
struct PositionCaseInsensitiveASCII
|
||||
{
|
||||
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
|
||||
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
|
||||
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
|
||||
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
|
||||
|
||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
|
||||
{
|
||||
return SearcherInBigHaystack(needle_data, needle_size);
|
||||
}
|
||||
|
||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||
{
|
||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||
}
|
||||
|
||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||
{
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||
|
||||
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
|
||||
};
|
||||
|
||||
struct PositionCaseSensitiveUTF8
|
||||
{
|
||||
using SearcherInBigHaystack = VolnitskyUTF8;
|
||||
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
|
||||
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
||||
|
||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||
{
|
||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||
}
|
||||
|
||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||
{
|
||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||
}
|
||||
|
||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||
{
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end)
|
||||
{
|
||||
size_t res = 0;
|
||||
for (auto it = begin; it != end; ++it)
|
||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
||||
++res;
|
||||
return res;
|
||||
}
|
||||
|
||||
static void toLowerIfNeed(std::string &) { }
|
||||
};
|
||||
|
||||
struct PositionCaseInsensitiveUTF8
|
||||
{
|
||||
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
|
||||
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
|
||||
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
|
||||
|
||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||
{
|
||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||
}
|
||||
|
||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||
{
|
||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||
}
|
||||
|
||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||
{
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end)
|
||||
{
|
||||
size_t res = 0;
|
||||
for (auto it = begin; it != end; ++it)
|
||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
||||
++res;
|
||||
return res;
|
||||
}
|
||||
|
||||
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
struct PositionImpl
|
||||
{
|
||||
static constexpr bool use_default_implementation_for_constants = false;
|
||||
|
||||
using ResultType = UInt64;
|
||||
|
||||
/// Find one substring in many strings.
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
const UInt8 * begin = data.data();
|
||||
const UInt8 * pos = begin;
|
||||
const UInt8 * end = pos + data.size();
|
||||
|
||||
/// Current index in the array of strings.
|
||||
size_t i = 0;
|
||||
|
||||
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
|
||||
|
||||
/// We will search for the next occurrence in all strings at once.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
{
|
||||
/// Determine which index it refers to.
|
||||
while (begin + offsets[i] <= pos)
|
||||
{
|
||||
res[i] = 0;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + needle.size() < begin + offsets[i])
|
||||
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
||||
else
|
||||
res[i] = 0;
|
||||
|
||||
pos = begin + offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
if (i < res.size())
|
||||
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
/// Search for substring in string.
|
||||
static void constantConstant(std::string data, std::string needle, UInt64 & res)
|
||||
{
|
||||
Impl::toLowerIfNeed(data);
|
||||
Impl::toLowerIfNeed(needle);
|
||||
|
||||
res = data.find(needle);
|
||||
if (res == std::string::npos)
|
||||
res = 0;
|
||||
else
|
||||
res = 1 + Impl::countChars(data.data(), data.data() + res);
|
||||
}
|
||||
|
||||
/// Search each time for a different single substring inside each time different string.
|
||||
static void vectorVector(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
ColumnString::Offset prev_haystack_offset = 0;
|
||||
ColumnString::Offset prev_needle_offset = 0;
|
||||
|
||||
size_t size = haystack_offsets.size();
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
|
||||
|
||||
if (0 == needle_size)
|
||||
{
|
||||
/// An empty string is always at the very beginning of `haystack`.
|
||||
res[i] = 1;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// It is assumed that the StringSearcher is not very difficult to initialize.
|
||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
|
||||
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
|
||||
|
||||
/// searcher returns a pointer to the found substring or to the end of `haystack`.
|
||||
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
|
||||
- &haystack_data[prev_haystack_offset];
|
||||
|
||||
if (pos != haystack_size)
|
||||
{
|
||||
res[i] = 1
|
||||
+ Impl::countChars(
|
||||
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
||||
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
|
||||
}
|
||||
else
|
||||
res[i] = 0;
|
||||
}
|
||||
|
||||
prev_haystack_offset = haystack_offsets[i];
|
||||
prev_needle_offset = needle_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/// Find many substrings in single string.
|
||||
static void constantVector(
|
||||
const String & haystack,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
// NOTE You could use haystack indexing. But this is a rare case.
|
||||
|
||||
ColumnString::Offset prev_needle_offset = 0;
|
||||
|
||||
size_t size = needle_offsets.size();
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||
|
||||
if (0 == needle_size)
|
||||
{
|
||||
res[i] = 1;
|
||||
}
|
||||
else
|
||||
{
|
||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
||||
|
||||
size_t pos = searcher.search(
|
||||
reinterpret_cast<const UInt8 *>(haystack.data()),
|
||||
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
|
||||
- reinterpret_cast<const UInt8 *>(haystack.data());
|
||||
|
||||
if (pos != haystack.size())
|
||||
{
|
||||
res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos);
|
||||
}
|
||||
else
|
||||
res[i] = 0;
|
||||
}
|
||||
|
||||
prev_needle_offset = needle_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
static void vectorFixedConstant(Args &&...)
|
||||
{
|
||||
throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
struct MultiSearchAllPositionsImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const std::vector<StringRef> & needles,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
||||
{
|
||||
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
||||
};
|
||||
|
||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||
|
||||
const size_t haystack_string_size = haystack_offsets.size();
|
||||
const size_t needles_size = needles.size();
|
||||
|
||||
/// Something can be uninitialized after the search itself
|
||||
std::fill(res.begin(), res.end(), 0);
|
||||
|
||||
while (searcher.hasMoreToSearch())
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size)
|
||||
{
|
||||
const auto * haystack = &haystack_data[prev_offset];
|
||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||
searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback);
|
||||
prev_offset = haystack_offsets[j];
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
struct MultiSearchImpl
|
||||
{
|
||||
using ResultType = UInt8;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
/// Variable for understanding, if we used offsets for the output, most
|
||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||
static constexpr bool is_column_array = false;
|
||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const std::vector<StringRef> & needles,
|
||||
PaddedPODArray<UInt8> & res,
|
||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||
{
|
||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||
const size_t haystack_string_size = haystack_offsets.size();
|
||||
res.resize(haystack_string_size);
|
||||
size_t iteration = 0;
|
||||
while (searcher.hasMoreToSearch())
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||
{
|
||||
const auto * haystack = &haystack_data[prev_offset];
|
||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||
if (iteration == 0 || !res[j])
|
||||
res[j] = searcher.searchOne(haystack, haystack_end);
|
||||
prev_offset = haystack_offsets[j];
|
||||
}
|
||||
++iteration;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
struct MultiSearchFirstPositionImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
/// Variable for understanding, if we used offsets for the output, most
|
||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||
static constexpr bool is_column_array = false;
|
||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const std::vector<StringRef> & needles,
|
||||
PaddedPODArray<UInt64> & res,
|
||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||
{
|
||||
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
||||
{
|
||||
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
||||
};
|
||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||
const size_t haystack_string_size = haystack_offsets.size();
|
||||
res.resize(haystack_string_size);
|
||||
size_t iteration = 0;
|
||||
while (searcher.hasMoreToSearch())
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||
{
|
||||
const auto * haystack = &haystack_data[prev_offset];
|
||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||
if (iteration == 0 || res[j] == 0)
|
||||
res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
||||
else
|
||||
{
|
||||
UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
||||
if (result != 0)
|
||||
res[j] = std::min(result, res[j]);
|
||||
}
|
||||
prev_offset = haystack_offsets[j];
|
||||
}
|
||||
++iteration;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
struct MultiSearchFirstIndexImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
/// Variable for understanding, if we used offsets for the output, most
|
||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||
static constexpr bool is_column_array = false;
|
||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const std::vector<StringRef> & needles,
|
||||
PaddedPODArray<UInt64> & res,
|
||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||
{
|
||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||
const size_t haystack_string_size = haystack_offsets.size();
|
||||
res.resize(haystack_string_size);
|
||||
size_t iteration = 0;
|
||||
while (searcher.hasMoreToSearch())
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||
{
|
||||
const auto * haystack = &haystack_data[prev_offset];
|
||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||
/// hasMoreToSearch traverse needles in increasing order
|
||||
if (iteration == 0 || res[j] == 0)
|
||||
res[j] = searcher.searchOneFirstIndex(haystack, haystack_end);
|
||||
prev_offset = haystack_offsets[j];
|
||||
}
|
||||
++iteration;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
|
||||
*/
|
||||
template <typename TokenSearcher, bool negate_result = false>
|
||||
struct HasTokenImpl
|
||||
{
|
||||
using ResultType = UInt8;
|
||||
|
||||
static constexpr bool use_default_implementation_for_constants = true;
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
if (offsets.empty())
|
||||
return;
|
||||
|
||||
const UInt8 * begin = data.data();
|
||||
const UInt8 * pos = begin;
|
||||
const UInt8 * end = pos + data.size();
|
||||
|
||||
/// The current index in the array of strings.
|
||||
size_t i = 0;
|
||||
|
||||
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
|
||||
|
||||
/// We will search for the next occurrence in all rows at once.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
{
|
||||
/// Let's determine which index it refers to.
|
||||
while (begin + offsets[i] <= pos)
|
||||
{
|
||||
res[i] = negate_result;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + pattern.size() < begin + offsets[i])
|
||||
res[i] = !negate_result;
|
||||
else
|
||||
res[i] = negate_result;
|
||||
|
||||
pos = begin + offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
static void vectorVector(Args &&...)
|
||||
{
|
||||
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
/// Search different needles in single haystack.
|
||||
template <typename... Args>
|
||||
static void constantVector(Args &&...)
|
||||
{
|
||||
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
static void vectorFixedConstant(Args &&...)
|
||||
{
|
||||
throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NamePosition
|
||||
{
|
||||
static constexpr auto name = "position";
|
||||
};
|
||||
struct NamePositionUTF8
|
||||
{
|
||||
static constexpr auto name = "positionUTF8";
|
||||
};
|
||||
struct NamePositionCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "positionCaseInsensitive";
|
||||
};
|
||||
struct NamePositionCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "positionCaseInsensitiveUTF8";
|
||||
};
|
||||
struct NameMultiSearchAllPositions
|
||||
{
|
||||
static constexpr auto name = "multiSearchAllPositions";
|
||||
};
|
||||
struct NameMultiSearchAllPositionsUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchAllPositionsUTF8";
|
||||
};
|
||||
struct NameMultiSearchAllPositionsCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
|
||||
};
|
||||
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
|
||||
};
|
||||
struct NameMultiSearchAny
|
||||
{
|
||||
static constexpr auto name = "multiSearchAny";
|
||||
};
|
||||
struct NameMultiSearchAnyUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchAnyUTF8";
|
||||
};
|
||||
struct NameMultiSearchAnyCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "multiSearchAnyCaseInsensitive";
|
||||
};
|
||||
struct NameMultiSearchAnyCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
|
||||
};
|
||||
struct NameMultiSearchFirstIndex
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstIndex";
|
||||
};
|
||||
struct NameMultiSearchFirstIndexUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstIndexUTF8";
|
||||
};
|
||||
struct NameMultiSearchFirstIndexCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
|
||||
};
|
||||
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
|
||||
};
|
||||
struct NameMultiSearchFirstPosition
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstPosition";
|
||||
};
|
||||
struct NameMultiSearchFirstPositionUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstPositionUTF8";
|
||||
};
|
||||
struct NameMultiSearchFirstPositionCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
|
||||
};
|
||||
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
|
||||
};
|
||||
|
||||
struct NameHasToken
|
||||
{
|
||||
static constexpr auto name = "hasToken";
|
||||
};
|
||||
|
||||
struct NameHasTokenCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "hasTokenCaseInsensitive";
|
||||
};
|
||||
|
||||
|
||||
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
|
||||
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
|
||||
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
|
||||
using FunctionPositionCaseInsensitiveUTF8
|
||||
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
|
||||
|
||||
using FunctionMultiSearchAllPositions
|
||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
|
||||
using FunctionMultiSearchAllPositionsUTF8
|
||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
|
||||
using FunctionMultiSearchAllPositionsCaseInsensitive
|
||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
|
||||
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
|
||||
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
|
||||
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
|
||||
|
||||
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
|
||||
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
|
||||
using FunctionMultiSearchCaseInsensitive
|
||||
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
|
||||
using FunctionMultiSearchCaseInsensitiveUTF8
|
||||
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
|
||||
|
||||
using FunctionMultiSearchFirstIndex
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
|
||||
using FunctionMultiSearchFirstIndexUTF8
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
|
||||
using FunctionMultiSearchFirstIndexCaseInsensitive
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
|
||||
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
|
||||
|
||||
using FunctionMultiSearchFirstPosition
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
|
||||
using FunctionMultiSearchFirstPositionUTF8
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
|
||||
using FunctionMultiSearchFirstPositionCaseInsensitive
|
||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
|
||||
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
|
||||
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
|
||||
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
|
||||
|
||||
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
|
||||
using FunctionHasTokenCaseInsensitive
|
||||
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
|
||||
|
||||
void registerFunctionsStringSearch(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionPositionUTF8>();
|
||||
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
||||
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionMultiSearchAllPositions>();
|
||||
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
|
||||
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
|
||||
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionMultiSearch>();
|
||||
factory.registerFunction<FunctionMultiSearchUTF8>();
|
||||
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
|
||||
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionMultiSearchFirstIndex>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionMultiSearchFirstPosition>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
|
||||
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionHasToken>();
|
||||
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
|
||||
|
||||
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
}
|
@ -176,7 +176,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
"Expected exactly one connection for shard " + toString(shard_info.shard_num), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// INSERT SELECT query returns empty block
|
||||
auto in_stream = std::make_shared<RemoteBlockInputStream>(*connections.front(), new_query_str, Block{}, context);
|
||||
auto in_stream = std::make_shared<RemoteBlockInputStream>(std::move(connections), new_query_str, Block{}, context);
|
||||
in_streams.push_back(in_stream);
|
||||
}
|
||||
out_streams.push_back(std::make_shared<NullBlockOutputStream>(Block()));
|
||||
|
@ -366,7 +366,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
||||
for (auto iterator = database->getTablesIterator(system_context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replica_names.emplace_back(iterator->table()->getStorageID());
|
||||
replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -97,10 +97,6 @@ function run_tests
|
||||
touch "$x"
|
||||
done
|
||||
|
||||
# FIXME remove some broken long tests
|
||||
rm "$test_prefix"/{IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format}.xml ||:
|
||||
|
||||
test_files=$(ls "$test_prefix"/*.xml)
|
||||
|
||||
# FIXME a quick crutch to bring the run time down for the unstable tests --
|
||||
# if some performance tests xmls were changed in a PR, run only these ones.
|
||||
@ -126,6 +122,17 @@ function run_tests
|
||||
test_files=$(ls "$test_prefix"/$CHPC_TEST_GLOB.xml)
|
||||
fi
|
||||
|
||||
if [ "$test_files" == "" ]
|
||||
then
|
||||
# FIXME remove some broken long tests
|
||||
for test_name in {IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format,arithmetic,cryptographic_hashes,logical_functions_{medium,small}}
|
||||
do
|
||||
printf "$test_name\tMarked as broken (see compare.sh)\n" >> skipped-tests.tsv
|
||||
rm "$test_prefix/$test_name.xml" ||:
|
||||
done
|
||||
test_files=$(ls "$test_prefix"/*.xml)
|
||||
fi
|
||||
|
||||
# Run the tests.
|
||||
test_name="<none>"
|
||||
for test in $test_files
|
||||
@ -275,9 +282,11 @@ create table test_times_tsv engine File(TSV, 'test-times.tsv') as
|
||||
from test_time join wall_clock using test
|
||||
order by avg_real_per_query desc;
|
||||
|
||||
create table all_queries_tsv engine File(TSV, 'all-queries.tsv') as
|
||||
select left, right, diff, rd, test, query
|
||||
from queries order by rd[3] desc;
|
||||
create table all_tests_tsv engine File(TSV, 'all-queries.tsv') as
|
||||
select left, right, diff,
|
||||
floor(left > right ? left / right : right / left, 3),
|
||||
rd, test, query
|
||||
from queries order by test, query;
|
||||
" 2> >(head -2 >> report-errors.rep) ||:
|
||||
|
||||
for version in {right,left}
|
||||
@ -429,6 +438,7 @@ case "$stage" in
|
||||
"report")
|
||||
time report ||:
|
||||
|
||||
time "$script_dir/report.py" --report=all-queries > all-queries.html 2> >(head -2 >> report-errors.rep) ||:
|
||||
time "$script_dir/report.py" > report.html
|
||||
;&
|
||||
esac
|
||||
|
@ -90,17 +90,23 @@ export PYTHONIOENCODING=utf-8
|
||||
# Use a default number of runs if not told otherwise
|
||||
export CHPC_RUNS=${CHPC_RUNS:-7}
|
||||
|
||||
# By default, use the main comparison script from the tested package, so that we
|
||||
# can change it in PRs.
|
||||
script_path="right/scripts"
|
||||
if [ -v CHPC_LOCAL_SCRIPT ]
|
||||
then
|
||||
script_path=".."
|
||||
fi
|
||||
|
||||
# Even if we have some errors, try our best to save the logs.
|
||||
set +e
|
||||
|
||||
# Use main comparison script from the tested package, so that we can change it
|
||||
# in PRs.
|
||||
# Older version use 'kill 0', so put the script into a separate process group
|
||||
# FIXME remove set +m in April 2020
|
||||
set +m
|
||||
{ \
|
||||
time ../download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \
|
||||
time stage=configure right/scripts/compare.sh ; \
|
||||
time stage=configure "$script_path"/compare.sh ; \
|
||||
} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee compare.log
|
||||
set -m
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/python3
|
||||
|
||||
import argparse
|
||||
import ast
|
||||
import collections
|
||||
import csv
|
||||
@ -8,6 +9,11 @@ import os
|
||||
import sys
|
||||
import traceback
|
||||
|
||||
parser = argparse.ArgumentParser(description='Create performance test report')
|
||||
parser.add_argument('--report', default='main', choices=['main', 'all-queries'],
|
||||
help='Which report to build')
|
||||
args = parser.parse_args()
|
||||
|
||||
report_errors = []
|
||||
error_tests = 0
|
||||
slow_average_tests = 0
|
||||
@ -16,7 +22,7 @@ slower_queries = 0
|
||||
unstable_queries = 0
|
||||
very_unstable_queries = 0
|
||||
|
||||
print("""
|
||||
header_template = """
|
||||
<!DOCTYPE html>
|
||||
<html>
|
||||
<style>
|
||||
@ -56,7 +62,7 @@ tr:nth-child(odd) td {{filter: brightness(95%);}}
|
||||
<div class="main">
|
||||
|
||||
<h1>ClickHouse performance comparison</h1>
|
||||
""".format())
|
||||
"""
|
||||
|
||||
table_anchor = 0
|
||||
row_anchor = 0
|
||||
@ -133,195 +139,252 @@ def printSimpleTable(caption, columns, rows):
|
||||
print(tableRow(row))
|
||||
print(tableEnd())
|
||||
|
||||
printSimpleTable('Tested commits', ['Old', 'New'],
|
||||
[['<pre>{}</pre>'.format(x) for x in
|
||||
[open('left-commit.txt').read(),
|
||||
open('right-commit.txt').read()]]])
|
||||
if args.report == 'main':
|
||||
print(header_template.format())
|
||||
|
||||
def print_changes():
|
||||
rows = tsvRows('changed-perf.tsv')
|
||||
if not rows:
|
||||
return
|
||||
printSimpleTable('Tested commits', ['Old', 'New'],
|
||||
[['<pre>{}</pre>'.format(x) for x in
|
||||
[open('left-commit.txt').read(),
|
||||
open('right-commit.txt').read()]]])
|
||||
|
||||
global faster_queries, slower_queries
|
||||
def print_changes():
|
||||
rows = tsvRows('changed-perf.tsv')
|
||||
if not rows:
|
||||
return
|
||||
|
||||
print(tableStart('Changes in performance'))
|
||||
columns = [
|
||||
'Old, s', # 0
|
||||
'New, s', # 1
|
||||
'Relative difference (new - old)/old', # 2
|
||||
'Randomization distribution quantiles \
|
||||
[5%, 50%, 95%, 99%]', # 3
|
||||
'Test', # 4
|
||||
'Query', # 5
|
||||
global faster_queries, slower_queries
|
||||
|
||||
print(tableStart('Changes in performance'))
|
||||
columns = [
|
||||
'Old, s', # 0
|
||||
'New, s', # 1
|
||||
'Relative difference (new - old)/old', # 2
|
||||
'Randomization distribution quantiles \
|
||||
[5%, 50%, 95%, 99%]', # 3
|
||||
'Test', # 4
|
||||
'Query', # 5
|
||||
]
|
||||
|
||||
print(tableHeader(columns))
|
||||
|
||||
attrs = ['' for c in columns]
|
||||
for row in rows:
|
||||
if float(row[2]) < 0.:
|
||||
faster_queries += 1
|
||||
attrs[2] = 'style="background: #adbdff"'
|
||||
else:
|
||||
slower_queries += 1
|
||||
attrs[2] = 'style="background: #ffb0a0"'
|
||||
|
||||
print(tableRow(row, attrs))
|
||||
|
||||
print(tableEnd())
|
||||
|
||||
print_changes()
|
||||
|
||||
slow_on_client_rows = tsvRows('slow-on-client.tsv')
|
||||
error_tests += len(slow_on_client_rows)
|
||||
printSimpleTable('Slow on client',
|
||||
['Client time, s', 'Server time, s', 'Ratio', 'Query'],
|
||||
slow_on_client_rows)
|
||||
|
||||
def print_unstable_queries():
|
||||
global unstable_queries
|
||||
global very_unstable_queries
|
||||
|
||||
unstable_rows = tsvRows('unstable-queries.tsv')
|
||||
if not unstable_rows:
|
||||
return
|
||||
|
||||
unstable_queries += len(unstable_rows)
|
||||
|
||||
columns = [
|
||||
'Old, s', #0
|
||||
'New, s', #1
|
||||
'Relative difference (new - old)/old', #2
|
||||
'Randomization distribution quantiles [5%, 50%, 95%, 99%]', #3
|
||||
'Test', #4
|
||||
'Query' #5
|
||||
]
|
||||
|
||||
print(tableHeader(columns))
|
||||
print(tableStart('Unstable queries'))
|
||||
print(tableHeader(columns))
|
||||
|
||||
attrs = ['' for c in columns]
|
||||
for row in rows:
|
||||
if float(row[2]) < 0.:
|
||||
faster_queries += 1
|
||||
attrs[2] = 'style="background: #adbdff"'
|
||||
else:
|
||||
slower_queries += 1
|
||||
attrs[2] = 'style="background: #ffb0a0"'
|
||||
attrs = ['' for c in columns]
|
||||
for r in unstable_rows:
|
||||
rd = ast.literal_eval(r[3])
|
||||
# Note the zero-based array index, this is rd[3] in SQL.
|
||||
if rd[2] > 0.2:
|
||||
very_unstable_queries += 1
|
||||
attrs[3] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[3] = ''
|
||||
|
||||
print(tableRow(row, attrs))
|
||||
print(tableRow(r, attrs))
|
||||
|
||||
print(tableEnd())
|
||||
print(tableEnd())
|
||||
|
||||
print_changes()
|
||||
print_unstable_queries()
|
||||
|
||||
slow_on_client_rows = tsvRows('slow-on-client.tsv')
|
||||
error_tests += len(slow_on_client_rows)
|
||||
printSimpleTable('Slow on client',
|
||||
['Client time, s', 'Server time, s', 'Ratio', 'Query'],
|
||||
slow_on_client_rows)
|
||||
run_error_rows = tsvRows('run-errors.tsv')
|
||||
error_tests += len(run_error_rows)
|
||||
printSimpleTable('Run errors', ['Test', 'Error'], run_error_rows)
|
||||
|
||||
def print_unstable_queries():
|
||||
global unstable_queries
|
||||
global very_unstable_queries
|
||||
skipped_tests_rows = tsvRows('skipped-tests.tsv')
|
||||
printSimpleTable('Skipped tests', ['Test', 'Reason'], skipped_tests_rows)
|
||||
|
||||
unstable_rows = tsvRows('unstable-queries.tsv')
|
||||
if not unstable_rows:
|
||||
return
|
||||
printSimpleTable('Tests with most unstable queries',
|
||||
['Test', 'Unstable', 'Changed perf', 'Total not OK'],
|
||||
tsvRows('bad-tests.tsv'))
|
||||
|
||||
unstable_queries += len(unstable_rows)
|
||||
def print_test_times():
|
||||
global slow_average_tests
|
||||
rows = tsvRows('test-times.tsv')
|
||||
if not rows:
|
||||
return
|
||||
|
||||
columns = [
|
||||
'Old, s', #0
|
||||
'New, s', #1
|
||||
'Relative difference (new - old)/old', #2
|
||||
'Randomization distribution quantiles [5%, 50%, 95%, 99%]', #3
|
||||
'Test', #4
|
||||
'Query' #5
|
||||
]
|
||||
columns = [
|
||||
'Test', #0
|
||||
'Wall clock time, s', #1
|
||||
'Total client time, s', #2
|
||||
'Total queries', #3
|
||||
'Ignored short queries', #4
|
||||
'Longest query<br>(sum for all runs), s', #5
|
||||
'Avg wall clock time<br>(sum for all runs), s', #6
|
||||
'Shortest query<br>(sum for all runs), s', #7
|
||||
]
|
||||
|
||||
print(tableStart('Unstable queries'))
|
||||
print(tableHeader(columns))
|
||||
print(tableStart('Test times'))
|
||||
print(tableHeader(columns))
|
||||
|
||||
attrs = ['' for c in columns]
|
||||
for r in unstable_rows:
|
||||
rd = ast.literal_eval(r[3])
|
||||
# Note the zero-based array index, this is rd[3] in SQL.
|
||||
if rd[2] > 0.2:
|
||||
very_unstable_queries += 1
|
||||
attrs[3] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[3] = ''
|
||||
attrs = ['' for c in columns]
|
||||
for r in rows:
|
||||
if float(r[6]) > 22:
|
||||
# FIXME should be 15s max -- investigate parallel_insert
|
||||
slow_average_tests += 1
|
||||
attrs[6] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[6] = ''
|
||||
|
||||
print(tableRow(r, attrs))
|
||||
if float(r[5]) > 30:
|
||||
slow_average_tests += 1
|
||||
attrs[5] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[5] = ''
|
||||
|
||||
print(tableEnd())
|
||||
print(tableRow(r, attrs))
|
||||
|
||||
print_unstable_queries()
|
||||
print(tableEnd())
|
||||
|
||||
run_error_rows = tsvRows('run-errors.tsv')
|
||||
error_tests += len(run_error_rows)
|
||||
printSimpleTable('Run errors', ['Test', 'Error'], run_error_rows)
|
||||
print_test_times()
|
||||
|
||||
skipped_tests_rows = tsvRows('skipped-tests.tsv')
|
||||
printSimpleTable('Skipped tests', ['Test', 'Reason'], skipped_tests_rows)
|
||||
|
||||
printSimpleTable('Tests with most unstable queries',
|
||||
['Test', 'Unstable', 'Changed perf', 'Total not OK'],
|
||||
tsvRows('bad-tests.tsv'))
|
||||
|
||||
def print_test_times():
|
||||
global slow_average_tests
|
||||
rows = tsvRows('test-times.tsv')
|
||||
if not rows:
|
||||
return
|
||||
|
||||
columns = [
|
||||
'Test', #0
|
||||
'Wall clock time, s', #1
|
||||
'Total client time, s', #2
|
||||
'Total queries', #3
|
||||
'Ignored short queries', #4
|
||||
'Longest query<br>(sum for all runs), s', #5
|
||||
'Avg wall clock time<br>(sum for all runs), s', #6
|
||||
'Shortest query<br>(sum for all runs), s', #7
|
||||
]
|
||||
|
||||
print(tableStart('Test times'))
|
||||
print(tableHeader(columns))
|
||||
|
||||
attrs = ['' for c in columns]
|
||||
for r in rows:
|
||||
if float(r[6]) > 22:
|
||||
# FIXME should be 15s max -- investigate parallel_insert
|
||||
slow_average_tests += 1
|
||||
attrs[6] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[6] = ''
|
||||
|
||||
if float(r[5]) > 30:
|
||||
slow_average_tests += 1
|
||||
attrs[5] = 'style="background: #ffb0a0"'
|
||||
else:
|
||||
attrs[5] = ''
|
||||
|
||||
print(tableRow(r, attrs))
|
||||
|
||||
print(tableEnd())
|
||||
|
||||
print_test_times()
|
||||
|
||||
# Add the errors reported by various steps of comparison script
|
||||
report_errors += [l.strip() for l in open('report-errors.rep')]
|
||||
if len(report_errors):
|
||||
print(tableStart('Errors while building the report'))
|
||||
print(tableHeader(['Error']))
|
||||
for x in report_errors:
|
||||
print(tableRow([x]))
|
||||
print(tableEnd())
|
||||
# Add the errors reported by various steps of comparison script
|
||||
report_errors += [l.strip() for l in open('report-errors.rep')]
|
||||
if len(report_errors):
|
||||
print(tableStart('Errors while building the report'))
|
||||
print(tableHeader(['Error']))
|
||||
for x in report_errors:
|
||||
print(tableRow([x]))
|
||||
print(tableEnd())
|
||||
|
||||
|
||||
print("""
|
||||
<p class="links">
|
||||
<a href="output.7z">Test output</a>
|
||||
<a href="compare.log">Log</a>
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
""")
|
||||
print("""
|
||||
<p class="links">
|
||||
<a href="output.7z">Test output</a>
|
||||
<a href="all-queries.html">All queries</a>
|
||||
<a href="compare.log">Log</a>
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
""")
|
||||
|
||||
status = 'success'
|
||||
message = 'See the report'
|
||||
message_array = []
|
||||
status = 'success'
|
||||
message = 'See the report'
|
||||
message_array = []
|
||||
|
||||
if slow_average_tests:
|
||||
status = 'failure'
|
||||
message_array.append(str(slow_average_tests) + ' too long')
|
||||
if slow_average_tests:
|
||||
status = 'failure'
|
||||
message_array.append(str(slow_average_tests) + ' too long')
|
||||
|
||||
if faster_queries:
|
||||
message_array.append(str(faster_queries) + ' faster')
|
||||
if faster_queries:
|
||||
message_array.append(str(faster_queries) + ' faster')
|
||||
|
||||
if slower_queries:
|
||||
status = 'failure'
|
||||
message_array.append(str(slower_queries) + ' slower')
|
||||
if slower_queries:
|
||||
status = 'failure'
|
||||
message_array.append(str(slower_queries) + ' slower')
|
||||
|
||||
if unstable_queries:
|
||||
message_array.append(str(unstable_queries) + ' unstable')
|
||||
if unstable_queries:
|
||||
message_array.append(str(unstable_queries) + ' unstable')
|
||||
|
||||
if very_unstable_queries:
|
||||
status = 'failure'
|
||||
if very_unstable_queries:
|
||||
status = 'failure'
|
||||
|
||||
error_tests += slow_average_tests
|
||||
if error_tests:
|
||||
status = 'failure'
|
||||
message_array.append(str(error_tests) + ' errors')
|
||||
error_tests += slow_average_tests
|
||||
if error_tests:
|
||||
status = 'failure'
|
||||
message_array.append(str(error_tests) + ' errors')
|
||||
|
||||
if message_array:
|
||||
message = ', '.join(message_array)
|
||||
if message_array:
|
||||
message = ', '.join(message_array)
|
||||
|
||||
if report_errors:
|
||||
status = 'failure'
|
||||
message = 'Errors while building the report.'
|
||||
if report_errors:
|
||||
status = 'failure'
|
||||
message = 'Errors while building the report.'
|
||||
|
||||
print("""
|
||||
<!--status: {status}-->
|
||||
<!--message: {message}-->
|
||||
""".format(status=status, message=message))
|
||||
print("""
|
||||
<!--status: {status}-->
|
||||
<!--message: {message}-->
|
||||
""".format(status=status, message=message))
|
||||
|
||||
elif args.report == 'all-queries':
|
||||
|
||||
print(header_template.format())
|
||||
|
||||
printSimpleTable('Tested commits', ['Old', 'New'],
|
||||
[['<pre>{}</pre>'.format(x) for x in
|
||||
[open('left-commit.txt').read(),
|
||||
open('right-commit.txt').read()]]])
|
||||
|
||||
def print_all_queries():
|
||||
rows = tsvRows('all-queries.tsv')
|
||||
if not rows:
|
||||
return
|
||||
|
||||
columns = [
|
||||
'Old, s', #0
|
||||
'New, s', #1
|
||||
'Relative difference (new - old)/old', #2
|
||||
'Times speedup/slowdown', #3
|
||||
'Randomization distribution quantiles \
|
||||
[5%, 50%, 95%, 99%]', #4
|
||||
'Test', #5
|
||||
'Query', #6
|
||||
]
|
||||
|
||||
print(tableStart('All query times'))
|
||||
print(tableHeader(columns))
|
||||
|
||||
attrs = ['' for c in columns]
|
||||
for r in rows:
|
||||
if float(r[2]) > 0.05:
|
||||
attrs[3] = 'style="background: #ffb0a0"'
|
||||
elif float(r[2]) < -0.05:
|
||||
attrs[3] = 'style="background: #adbdff"'
|
||||
else:
|
||||
attrs[3] = ''
|
||||
|
||||
print(tableRow(r, attrs))
|
||||
|
||||
print(tableEnd())
|
||||
|
||||
print_all_queries()
|
||||
|
||||
print("""
|
||||
<p class="links">
|
||||
<a href="output.7z">Test output</a>
|
||||
<a href="report.html">Main report</a>
|
||||
<a href="compare.log">Log</a>
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
""")
|
||||
|
@ -402,18 +402,128 @@ SELECT
|
||||
└───────────────────────────┴──────────────────────────────┘
|
||||
```
|
||||
|
||||
## parseDateTimeBestEffort {#type_conversion_functions-parsedatetimebesteffort}
|
||||
## parseDateTimeBestEffort {#parsedatetimebesteffort}
|
||||
|
||||
Parse a number type argument to a Date or DateTime type.
|
||||
different from toDate and toDateTime, parseDateTimeBestEffort can progress more complex date format.
|
||||
For more information, see the link: [Complex Date Format](https://xkcd.com/1179/)
|
||||
Converts a date and time in the [String](../../data_types/string.md) representation to [DateTime](../../data_types/datetime.md#data_type-datetime) data type.
|
||||
|
||||
The function parses [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 1123 - 5.2.14 RFC-822 Date and Time Specification](https://tools.ietf.org/html/rfc1123#page-55), ClickHouse's and some other date and time formats.
|
||||
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
parseDateTimeBestEffort(time_string [, time_zone]);
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `time_string` — String containing a date and time to convert. [String](../../data_types/string.md).
|
||||
- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../data_types/string.md).
|
||||
|
||||
|
||||
**Supported non-standard formats**
|
||||
|
||||
- A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time).
|
||||
- A string with a date and a time component: `YYYYMMDDhhmmss`, `DD/MM/YYYY hh:mm:ss`, `DD-MM-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc.
|
||||
- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` etc.
|
||||
- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `YYYY-MM` are substituted as `2000-01`.
|
||||
- A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`.
|
||||
|
||||
For all of the formats with separator the function parses months names expressed by their full name or by the first three letters of a month name. Examples: `24/DEC/18`, `24-Dec-18`, `01-September-2018`.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- `time_string` converted to the `DateTime` data type.
|
||||
|
||||
**Examples**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('12/12/2020 12:12:57')
|
||||
AS parseDateTimeBestEffort;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2020-12-12 12:12:57 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2018-08-18 10:22:16 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('1284101485')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2015-07-07 12:04:41 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('2018-12-12 10:12:12')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2018-12-12 10:12:12 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('10 20:19')
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort('10 20:19')─┐
|
||||
│ 2000-01-10 20:19:00 │
|
||||
└─────────────────────────────────────┘
|
||||
```
|
||||
|
||||
**See Also**
|
||||
|
||||
- [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/)
|
||||
- [RFC 1123](https://tools.ietf.org/html/rfc1123)
|
||||
- [toDate](#todate)
|
||||
- [toDateTime](#todatetime)
|
||||
|
||||
## parseDateTimeBestEffortOrNull {#parsedatetimebesteffortornull}
|
||||
|
||||
Same as for [parseDateTimeBestEffort](#type_conversion_functions-parsedatetimebesteffort) except that it returns null when it encounters a date format that cannot be processed.
|
||||
Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns null when it encounters a date format that cannot be processed.
|
||||
|
||||
## parseDateTimeBestEffortOrZero {#parsedatetimebesteffortorzero}
|
||||
|
||||
Same as for [parseDateTimeBestEffort](#type_conversion_functions-parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed.
|
||||
Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed.
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/functions/type_conversion_functions/) <!--hide-->
|
||||
|
@ -35,7 +35,7 @@
|
||||
- [StripeLog](stripelog.md)
|
||||
- [Log](log.md)
|
||||
|
||||
### Движки для интергации {#dvizhki-dlia-intergatsii}
|
||||
### Движки для интеграции {#dvizhki-dlia-integratsii}
|
||||
|
||||
Движки для связи с другими системами хранения и обработки данных.
|
||||
|
||||
|
@ -360,7 +360,7 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null
|
||||
|
||||
Приводит аргумент из числового типа данных к типу данных [IntervalType](../../data_types/special_data_types/interval.md).
|
||||
|
||||
**Синтксис**
|
||||
**Синтаксис**
|
||||
|
||||
``` sql
|
||||
toIntervalSecond(number)
|
||||
@ -399,4 +399,120 @@ SELECT
|
||||
└───────────────────────────┴──────────────────────────────┘
|
||||
```
|
||||
|
||||
## parseDateTimeBestEffort {#parsedatetimebesteffort}
|
||||
|
||||
Преобразует дату и время в [строковом](../../data_types/string.md) представлении к типу данных [DateTime](../../data_types/datetime.md#data_type-datetime).
|
||||
|
||||
Функция распознаёт форматы [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 1123 - 5.2.14 RFC-822 Date and Time Specification](https://tools.ietf.org/html/rfc1123#page-55), формат даты времени ClickHouse's а также некоторые другие форматы.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
parseDateTimeBestEffort(time_string[, time_zone]);
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `time_string` — строка, содержащая дату и время для преобразования. [String](../../data_types/string.md).
|
||||
- `time_zone` — часовой пояс. Функция анализирует `time_string` в соответствии с заданным часовым поясом. [String](../../data_types/string.md).
|
||||
|
||||
**Поддерживаемые нестандартные форматы**
|
||||
|
||||
- [Unix timestamp](https://ru.wikipedia.org/wiki/Unix-время) в строковом представлении. 9 или 10 символов.
|
||||
- Строка с датой и временем: `YYYYMMDDhhmmss`, `DD/MM/YYYY hh:mm:ss`, `DD-MM-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc.
|
||||
- Строка с датой, но без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` и т.д.
|
||||
- Строка с временем, и с днём: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` принимается равным `2000-01`.
|
||||
- Строка, содержащая дату и время вместе с информацией о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm`, и т.д. Например, `2020-12-12 17:36:00 -5:00`.
|
||||
|
||||
Для всех форматов с разделителями функция распознаёт названия месяцев, выраженных в виде полного англоязычного имени месяца или в виде первых трёх символов имени месяца. Примеры: `24/DEC/18`, `24-Dec-18`, `01-September-2018`.
|
||||
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- `time_string` преобразованная к типу данных `DateTime`.
|
||||
|
||||
**Примеры**
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('12/12/2020 12:12:57')
|
||||
AS parseDateTimeBestEffort;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2020-12-12 12:12:57 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2018-08-18 10:22:16 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('1284101485')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2015-07-07 12:04:41 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('2018-12-12 10:12:12')
|
||||
AS parseDateTimeBestEffort
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort─┐
|
||||
│ 2018-12-12 10:12:12 │
|
||||
└─────────────────────────┘
|
||||
```
|
||||
|
||||
Запрос:
|
||||
|
||||
```sql
|
||||
SELECT parseDateTimeBestEffort('10 20:19')
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```text
|
||||
┌─parseDateTimeBestEffort('10 20:19')─┐
|
||||
│ 2000-01-10 20:19:00 │
|
||||
└─────────────────────────────────────┘
|
||||
```
|
||||
|
||||
**См. также**
|
||||
|
||||
- [Информация о формате ISO 8601 от @xkcd](https://xkcd.com/1179/)
|
||||
- [RFC 1123](https://tools.ietf.org/html/rfc1123)
|
||||
- [toDate](#todate)
|
||||
- [toDateTime](#todatetime)
|
||||
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/type_conversion_functions/) <!--hide-->
|
||||
|
File diff suppressed because one or more lines are too long
@ -1,16 +1,8 @@
|
||||
<test>
|
||||
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<total_time_ms>10000</total_time_ms>
|
||||
</all_of>
|
||||
</stop_conditions>
|
||||
|
||||
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) >= 0), range(100000000)), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) >= 2), range(100000000)), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 2), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) = 0), range(100000000)), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) = 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10)</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) >= 0), range(100000000)), 1, 10) FORMAT Null</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) >= 2), range(100000000)), 1, 10) FORMAT Null</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 2), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> ((x % 2) = 0), range(100000000)), 1, 10) FORMAT Null</query>
|
||||
<query>SELECT arraySlice(arrayFill(x -> (((x.1) % 2) = 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null</query>
|
||||
</test>
|
||||
|
@ -1,17 +1,4 @@
|
||||
<test>
|
||||
|
||||
<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>
|
||||
|
||||
|
||||
<preconditions>
|
||||
<table_exists>test.hits</table_exists>
|
||||
</preconditions>
|
||||
@ -24,13 +11,13 @@
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat(MobilePhoneModel, 'Hello'))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat(PageCharset, 'a'))</query>
|
||||
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}{}', URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}{}', URL, SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}{}', MobilePhoneModel, SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}Hello', URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('World{}', SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}Hello', MobilePhoneModel))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}a', PageCharset))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}{{}}', URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}{{}}', URL, SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}{{}}', MobilePhoneModel, SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}Hello', URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('World{{}}', SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}Hello', MobilePhoneModel))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}a', PageCharset))</query>
|
||||
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat(URL, URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat(URL, SearchPhrase, MobilePhoneModel))</query>
|
||||
@ -39,10 +26,10 @@
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat(MobilePhoneModel, 'Hello', PageCharset))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(concat('a', PageCharset, 'b'))</query>
|
||||
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}{}{}', URL, URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}{}{}', URL, SearchPhrase, MobilePhoneModel))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}Hello{}', URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('Hello{}World', SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{}Hello{}', MobilePhoneModel, PageCharset))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('a{}b', PageCharset))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}{{}}{{}}', URL, URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}{{}}{{}}', URL, SearchPhrase, MobilePhoneModel))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}Hello{{}}', URL, URL))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('Hello{{}}World', SearchPhrase))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('{{}}Hello{{}}', MobilePhoneModel, PageCharset))</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(format('a{{}}b', PageCharset))</query>
|
||||
</test>
|
||||
|
Loading…
Reference in New Issue
Block a user