mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Review fixes
This commit is contained in:
parent
772bb0b70b
commit
3881a85c39
@ -30,61 +30,67 @@ namespace DB
|
||||
}
|
||||
}
|
||||
|
||||
static constexpr size_t MAX_LANG_LENGTH = 128;
|
||||
|
||||
std::unique_ptr<AvailableCollationLocales> AvailableCollationLocales::instance_impl;
|
||||
std::once_flag AvailableCollationLocales::init_flag;
|
||||
|
||||
void AvailableCollationLocales::init()
|
||||
AvailableCollationLocales::AvailableCollationLocales()
|
||||
{
|
||||
instance_impl = std::make_unique<AvailableCollationLocales>();
|
||||
#if USE_ICU
|
||||
size_t available_locales_count = ucol_countAvailable();
|
||||
for (size_t i = 0; i < available_locales_count; ++i)
|
||||
{
|
||||
std::string locale_name = ucol_getAvailable(i);
|
||||
UChar lang_buffer[128]; /// 128 is enough for language name
|
||||
char normal_buf[128];
|
||||
UChar lang_buffer[MAX_LANG_LENGTH];
|
||||
char normal_buf[MAX_LANG_LENGTH];
|
||||
UErrorCode status = U_ZERO_ERROR;
|
||||
|
||||
/// All names will be in English language
|
||||
size_t lang_length = uloc_getDisplayLanguage(locale_name.c_str(), "en", lang_buffer, 128, &status);
|
||||
if (U_FAILURE(status))
|
||||
instance_impl->available_collation_locales.push_back(LocaleAndLanguage{locale_name, "unknown"});
|
||||
else
|
||||
size_t lang_length = uloc_getDisplayLanguage(
|
||||
locale_name.c_str(), "en", lang_buffer, MAX_LANG_LENGTH, &status);
|
||||
std::optional<std::string> lang;
|
||||
|
||||
if (!U_FAILURE(status))
|
||||
{
|
||||
/// Convert language name from UChar array to normal char array.
|
||||
/// We use English language for name, so all UChar's length is equal to sizeof(char)
|
||||
u_UCharsToChars(lang_buffer, normal_buf, lang_length);
|
||||
LocaleAndLanguage result{locale_name, std::string(normal_buf, lang_length)};
|
||||
instance_impl->available_collation_locales.push_back(result);
|
||||
lang.emplace(std::string(normal_buf, lang_length));
|
||||
}
|
||||
|
||||
locales_map.emplace(Poco::toLower(locale_name), LocaleAndLanguage{locale_name, lang});
|
||||
}
|
||||
|
||||
auto comparator = [] (const LocaleAndLanguage & f, const LocaleAndLanguage & s) { return f.locale_name < s.locale_name; };
|
||||
std::sort(instance_impl->available_collation_locales.begin(), instance_impl->available_collation_locales.end(), comparator);
|
||||
#endif
|
||||
}
|
||||
|
||||
AvailableCollationLocales & AvailableCollationLocales::instance()
|
||||
const AvailableCollationLocales & AvailableCollationLocales::instance()
|
||||
{
|
||||
std::call_once(init_flag, AvailableCollationLocales::init);
|
||||
return *instance_impl;
|
||||
static AvailableCollationLocales instance;
|
||||
return instance;
|
||||
}
|
||||
|
||||
const std::vector<AvailableCollationLocales::LocaleAndLanguage> & AvailableCollationLocales::getAvailableCollations() const
|
||||
AvailableCollationLocales::LocalesVector AvailableCollationLocales::getAvailableCollations() const
|
||||
{
|
||||
return available_collation_locales;
|
||||
LocalesVector result;
|
||||
for (const auto & name_and_locale : locales_map)
|
||||
result.push_back(name_and_locale.second);
|
||||
|
||||
auto comparator = [] (const LocaleAndLanguage & f, const LocaleAndLanguage & s)
|
||||
{
|
||||
return f.locale_name < s.locale_name;
|
||||
};
|
||||
std::sort(result.begin(), result.end(), comparator);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
bool AvailableCollationLocales::isCollationSupported(const std::string & s) const
|
||||
bool AvailableCollationLocales::isCollationSupported(const std::string & locale_name) const
|
||||
{
|
||||
std::string lower = Poco::toLower(s);
|
||||
for (const auto & locale_and_lang : available_collation_locales)
|
||||
{
|
||||
if (lower == Poco::toLower(locale_and_lang.locale_name))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
/// We support locale names in any case, so we have to convert all to lower case
|
||||
return locales_map.count(Poco::toLower(locale_name));
|
||||
}
|
||||
|
||||
Collator::Collator(const std::string & locale_) : locale(Poco::toLower(locale_))
|
||||
Collator::Collator(const std::string & locale_)
|
||||
: locale(Poco::toLower(locale_))
|
||||
{
|
||||
#if USE_ICU
|
||||
/// We check it here, because ucol_open will fallback to default locale for
|
||||
|
@ -3,8 +3,8 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
struct UCollator;
|
||||
|
||||
@ -15,24 +15,25 @@ public:
|
||||
|
||||
struct LocaleAndLanguage
|
||||
{
|
||||
std::string locale_name; /// in ISO format
|
||||
std::string language; /// in English
|
||||
std::string locale_name; /// ISO locale code
|
||||
std::optional<std::string> language; /// full language name in English
|
||||
};
|
||||
|
||||
static AvailableCollationLocales & instance();
|
||||
using AvailableLocalesMap = std::unordered_map<std::string, LocaleAndLanguage>;
|
||||
using LocalesVector = std::vector<LocaleAndLanguage>;
|
||||
|
||||
/// Get all collations with names
|
||||
const std::vector<LocaleAndLanguage> & getAvailableCollations() const;
|
||||
static const AvailableCollationLocales & instance();
|
||||
|
||||
/// Get all collations with names in sorted order
|
||||
LocalesVector getAvailableCollations() const;
|
||||
|
||||
/// Check that collation is supported
|
||||
bool isCollationSupported(const std::string & s) const;
|
||||
bool isCollationSupported(const std::string & locale_name) const;
|
||||
|
||||
private:
|
||||
static std::once_flag init_flag;
|
||||
static std::unique_ptr<AvailableCollationLocales> instance_impl;
|
||||
static void init();
|
||||
AvailableCollationLocales();
|
||||
private:
|
||||
std::vector<LocaleAndLanguage> available_collation_locales;
|
||||
AvailableLocalesMap locales_map;
|
||||
};
|
||||
|
||||
class Collator : private boost::noncopyable
|
||||
|
@ -52,7 +52,7 @@ struct Less
|
||||
{
|
||||
for (auto it = left_columns.begin(), jt = right_columns.begin(); it != left_columns.end(); ++it, ++jt)
|
||||
{
|
||||
int res = it->second.direction * it->first->compareAt(a, b, *jt->first, it->second.nulls_direction);
|
||||
int res = it->description.direction * it->column->compareAt(a, b, *jt->column, it->description.nulls_direction);
|
||||
if (res < 0)
|
||||
return true;
|
||||
else if (res > 0)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <pdqsort.h>
|
||||
|
||||
@ -14,22 +15,9 @@ namespace ErrorCodes
|
||||
extern const int BAD_COLLATION;
|
||||
}
|
||||
|
||||
|
||||
static inline const IColumn * needCollation(const IColumn * column, const SortColumnDescription & description)
|
||||
static bool isCollationRequired(const SortColumnDescription & description)
|
||||
{
|
||||
if (!description.collator)
|
||||
return nullptr;
|
||||
|
||||
auto column_result = column;
|
||||
if (auto const_column = typeid_cast<const ColumnConst *>(column))
|
||||
column_result = &const_column->getDataColumn();
|
||||
|
||||
if (typeid_cast<const ColumnString *>(column_result))
|
||||
return column_result;
|
||||
|
||||
/// TODO Nullable(String)
|
||||
|
||||
throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION);
|
||||
return description.collator != nullptr;
|
||||
}
|
||||
|
||||
|
||||
@ -45,7 +33,7 @@ ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, c
|
||||
? block.getByName(description[i].column_name).column.get()
|
||||
: block.safeGetByPosition(description[i].column_number).column.get();
|
||||
|
||||
res.emplace_back(column, description[i]);
|
||||
res.emplace_back(ColumnWithSortDescription{column, description[i], isColumnConst(*column)});
|
||||
}
|
||||
|
||||
return res;
|
||||
@ -62,7 +50,11 @@ struct PartialSortingLess
|
||||
{
|
||||
for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
||||
{
|
||||
int res = it->second.direction * it->first->compareAt(a, b, *it->first, it->second.nulls_direction);
|
||||
int res;
|
||||
if (it->column_const)
|
||||
res = 0;
|
||||
else
|
||||
res = it->description.direction * it->column->compareAt(a, b, *it->column, it->description.nulls_direction);
|
||||
if (res < 0)
|
||||
return true;
|
||||
else if (res > 0)
|
||||
@ -77,22 +69,28 @@ struct PartialSortingLessWithCollation
|
||||
{
|
||||
const ColumnsWithSortDescriptions & columns;
|
||||
|
||||
explicit PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
||||
explicit PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_)
|
||||
: columns(columns_)
|
||||
{
|
||||
}
|
||||
|
||||
bool operator() (size_t a, size_t b) const
|
||||
{
|
||||
for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
||||
{
|
||||
int res;
|
||||
if (auto column_string_ptr = needCollation(it->first, it->second))
|
||||
|
||||
if (it->column_const)
|
||||
res = 0;
|
||||
else if (isCollationRequired(it->description))
|
||||
{
|
||||
const ColumnString & column_string = typeid_cast<const ColumnString &>(*column_string_ptr);
|
||||
res = column_string.compareAtWithCollation(a, b, *it->first, *it->second.collator);
|
||||
const ColumnString & column_string = assert_cast<const ColumnString &>(*it->column);
|
||||
res = column_string.compareAtWithCollation(a, b, *it->column, *it->description.collator);
|
||||
|
||||
}
|
||||
else
|
||||
res = it->first->compareAt(a, b, *it->first, it->second.nulls_direction);
|
||||
|
||||
res *= it->second.direction;
|
||||
res = it->column->compareAt(a, b, *it->column, it->description.nulls_direction);
|
||||
res *= it->description.direction;
|
||||
if (res < 0)
|
||||
return true;
|
||||
else if (res > 0)
|
||||
@ -107,6 +105,14 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
|
||||
if (!block)
|
||||
return;
|
||||
|
||||
size_t size = block.rows();
|
||||
IColumn::Permutation perm(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
perm[i] = i;
|
||||
|
||||
if (limit >= size)
|
||||
limit = 0;
|
||||
|
||||
/// If only one column to sort by
|
||||
if (description.size() == 1)
|
||||
{
|
||||
@ -116,14 +122,18 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
|
||||
? block.getByName(description[0].column_name).column.get()
|
||||
: block.safeGetByPosition(description[0].column_number).column.get();
|
||||
|
||||
IColumn::Permutation perm;
|
||||
if (auto column_string_ptr = needCollation(column, description[0]))
|
||||
if (isCollationRequired(description[0]))
|
||||
{
|
||||
const ColumnString & column_string = typeid_cast<const ColumnString &>(*column_string_ptr);
|
||||
column_string.getPermutationWithCollation(*description[0].collator, reverse, limit, perm);
|
||||
/// it it's real string column, than we need sort
|
||||
if (const ColumnString * column_string = checkAndGetColumn<ColumnString>(column))
|
||||
column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm);
|
||||
else if (!checkAndGetColumnConstData<ColumnString>(column))
|
||||
throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION);
|
||||
|
||||
}
|
||||
else
|
||||
else if (!isColumnConst(*column))
|
||||
column->getPermutation(reverse, limit, description[0].nulls_direction, perm);
|
||||
/// we don't need to do anything with const column
|
||||
|
||||
size_t columns = block.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
@ -131,23 +141,18 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t size = block.rows();
|
||||
IColumn::Permutation perm(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
perm[i] = i;
|
||||
|
||||
if (limit >= size)
|
||||
limit = 0;
|
||||
|
||||
bool need_collation = false;
|
||||
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
||||
|
||||
for (size_t i = 0, num_sort_columns = description.size(); i < num_sort_columns; ++i)
|
||||
{
|
||||
if (needCollation(columns_with_sort_desc[i].first, description[i]))
|
||||
const IColumn * column = columns_with_sort_desc[i].column;
|
||||
if (isCollationRequired(description[i]))
|
||||
{
|
||||
if (!checkAndGetColumn<ColumnString>(column) && !checkAndGetColumnConstData<ColumnString>(column))
|
||||
throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION);
|
||||
|
||||
need_collation = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@ -172,7 +177,9 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
|
||||
|
||||
size_t columns = block.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -29,7 +29,17 @@ void stableGetPermutation(const Block & block, const SortDescription & descripti
|
||||
*/
|
||||
bool isAlreadySorted(const Block & block, const SortDescription & description);
|
||||
|
||||
using ColumnsWithSortDescriptions = std::vector<std::pair<const IColumn *, SortColumnDescription>>;
|
||||
/// Column with description for sort
|
||||
struct ColumnWithSortDescription
|
||||
{
|
||||
const IColumn * column;
|
||||
SortColumnDescription description;
|
||||
|
||||
/// It means, that this column is ColumnConst
|
||||
bool column_const = false;
|
||||
};
|
||||
|
||||
using ColumnsWithSortDescriptions = std::vector<ColumnWithSortDescription>;
|
||||
|
||||
ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Columns/Collator.h>
|
||||
#include <Storages/System/StorageSystemCollations.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -8,7 +9,7 @@ NamesAndTypesList StorageSystemCollations::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"language", std::make_shared<DataTypeString>()},
|
||||
{"language", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
};
|
||||
}
|
||||
|
||||
@ -17,7 +18,7 @@ void StorageSystemCollations::fillData(MutableColumns & res_columns, const Conte
|
||||
for (const auto & [locale, lang]: AvailableCollationLocales::instance().getAvailableCollations())
|
||||
{
|
||||
res_columns[0]->insert(locale);
|
||||
res_columns[1]->insert(lang);
|
||||
res_columns[1]->insert(lang ? *lang : Field());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -293,3 +293,7 @@ z
|
||||
я 2
|
||||
Я 5
|
||||
ζ
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
|
@ -36,6 +36,11 @@ SELECT x, n FROM (SELECT ['а', 'я', 'ё', 'А', 'Я', 'Ё'] AS arr) ARRAY JOIN
|
||||
--- Const expression
|
||||
SELECT 'ζ' as x ORDER BY x COLLATE 'el';
|
||||
|
||||
-- check order by const with collation
|
||||
SELECT number FROM numbers(2) ORDER BY 'x' COLLATE 'el';
|
||||
|
||||
-- check const and non const columns in order
|
||||
SELECT number FROM numbers(2) ORDER BY 'x', toString(number), 'y' COLLATE 'el';
|
||||
|
||||
--- Trash locales
|
||||
SELECT '' as x ORDER BY x COLLATE 'qq'; --{serverError 186}
|
||||
|
Loading…
Reference in New Issue
Block a user