Merge branch 'master' into alter_rename_column

This commit is contained in:
alesapin 2020-04-03 12:00:15 +03:00
commit 4902346f86
13 changed files with 242 additions and 726 deletions

View File

@ -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)
{

View File

@ -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

View File

@ -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);
}
}

View File

@ -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()));

View File

@ -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()});
}
}

View File

@ -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-->

View File

@ -35,7 +35,7 @@
- [StripeLog](stripelog.md)
- [Log](log.md)
### Движки для интергации {#dvizhki-dlia-intergatsii}
### Движки для интеграции {#dvizhki-dlia-integratsii}
Движки для связи с другими системами хранения и обработки данных.

View File

@ -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-->

View File

@ -1 +1 @@
../../dbms/tests/config/decimals_dictionary.xml
config/decimals_dictionary.xml

View File

@ -1 +1 @@
../../../docs/en/development/developer_instruction.md
../../docs/en/development/developer_instruction.md

View File

@ -1 +1 @@
../../../docs/ru/development/developer_instruction.md
../../docs/ru/development/developer_instruction.md

View File

@ -1 +1 @@
../../dbms/tests/config/ints_dictionary.xml
config/ints_dictionary.xml

View File

@ -1 +1 @@
../../dbms/tests/config/strings_dictionary.xml
config/strings_dictionary.xml