Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into g-arslan-add-sqlite-support

This commit is contained in:
kssenii 2021-07-10 06:50:46 +00:00
commit 40281f9283
27 changed files with 265 additions and 79 deletions

View File

@ -2,11 +2,11 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
SET(VERSION_REVISION 54453)
SET(VERSION_REVISION 54454)
SET(VERSION_MAJOR 21)
SET(VERSION_MINOR 8)
SET(VERSION_MINOR 9)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH fb895056568e26200629c7d19626e92d2dedc70d)
SET(VERSION_DESCRIBE v21.8.1.1-prestable)
SET(VERSION_STRING 21.8.1.1)
SET(VERSION_GITHASH f48c5af90c2ad51955d1ee3b6b05d006b03e4238)
SET(VERSION_DESCRIBE v21.9.1.1-prestable)
SET(VERSION_STRING 21.9.1.1)
# end of autochange

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (21.8.1.1) unstable; urgency=low
clickhouse (21.9.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 28 Jun 2021 00:50:15 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Sat, 10 Jul 2021 08:22:49 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.8.1.*
ARG version=21.9.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.8.1.*
ARG version=21.9.1.*
ARG gosu_ver=1.10
# set non-empty deb_location_url url to create a docker image

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.8.1.*
ARG version=21.9.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -12,7 +12,7 @@ For information on connecting and configuring external dictionaries, see [Extern
## dictGet, dictGetOrDefault, dictGetOrNull {#dictget}
Retrieves values from an external dictionary.
Retrieves values from an external dictionary.
``` sql
dictGet('dict_name', attr_names, id_expr)
@ -24,7 +24,7 @@ dictGetOrNull('dict_name', attr_name, id_expr)
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `attr_names` — Name of the column of the dictionary, [String literal](../../sql-reference/syntax.md#syntax-string-literal), or tuple of column names, [Tuple](../../sql-reference/data-types/tuple.md)([String literal](../../sql-reference/syntax.md#syntax-string-literal)).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md) or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning dictionary key-type value or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
- `default_value_expr` — Values returned if the dictionary does not contain a row with the `id_expr` key. [Expression](../../sql-reference/syntax.md#syntax-expressions) or [Tuple](../../sql-reference/data-types/tuple.md)([Expression](../../sql-reference/syntax.md#syntax-expressions)), returning the value (or values) in the data types configured for the `attr_names` attribute.
**Returned value**
@ -138,7 +138,7 @@ Configure the external dictionary:
<name>c2</name>
<type>String</type>
<null_value></null_value>
</attribute>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
@ -237,7 +237,7 @@ dictHas('dict_name', id_expr)
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md) or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning dictionary key-type value or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
**Returned value**
@ -292,16 +292,16 @@ Type: `UInt8`.
Returns first-level children as an array of indexes. It is the inverse transformation for [dictGetHierarchy](#dictgethierarchy).
**Syntax**
**Syntax**
``` sql
dictGetChildren(dict_name, key)
```
**Arguments**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `key` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `key` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
**Returned values**
@ -339,7 +339,7 @@ SELECT dictGetChildren('hierarchy_flat_dictionary', number) FROM system.numbers
## dictGetDescendant {#dictgetdescendant}
Returns all descendants as if [dictGetChildren](#dictgetchildren) function was applied `level` times recursively.
Returns all descendants as if [dictGetChildren](#dictgetchildren) function was applied `level` times recursively.
**Syntax**
@ -347,9 +347,9 @@ Returns all descendants as if [dictGetChildren](#dictgetchildren) function was a
dictGetDescendants(dict_name, key, level)
```
**Arguments**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `key` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
- `level` — Hierarchy level. If `level = 0` returns all descendants to the end. [UInt8](../../sql-reference/data-types/int-uint.md).

View File

@ -430,6 +430,7 @@ private:
{TokenType::ClosingRoundBracket, Replxx::Color::BROWN},
{TokenType::OpeningSquareBracket, Replxx::Color::BROWN},
{TokenType::ClosingSquareBracket, Replxx::Color::BROWN},
{TokenType::DoubleColon, Replxx::Color::BROWN},
{TokenType::OpeningCurlyBrace, Replxx::Color::INTENSE},
{TokenType::ClosingCurlyBrace, Replxx::Color::INTENSE},

View File

@ -10,16 +10,10 @@ namespace fs = std::filesystem;
namespace DB
{
/// Checks if file exists without throwing an exception but with message in console.
bool safeFsExists(const auto & path)
bool safeFsExists(const String & path)
{
std::error_code ec;
bool res = fs::exists(path, ec);
if (ec)
{
std::cerr << "Can't check '" << path << "': [" << ec.value() << "] " << ec.message() << std::endl;
}
return res;
return fs::exists(path, ec);
};
bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path)

View File

@ -1,11 +1,13 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <Common/assert_cast.h>
@ -39,6 +41,11 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
return std::make_shared<DataTypeTuple>(elements);
}
if (const auto * map_type = typeid_cast<const DataTypeMap *>(type.get()))
{
return std::make_shared<DataTypeMap>(recursiveRemoveLowCardinality(map_type->getKeyType()), recursiveRemoveLowCardinality(map_type->getValueType()));
}
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
return low_cardinality_type->getDictionaryType();
@ -78,6 +85,16 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
return ColumnTuple::create(columns);
}
if (const auto * column_map = typeid_cast<const ColumnMap *>(column.get()))
{
const auto & nested = column_map->getNestedColumnPtr();
auto nested_no_lc = recursiveRemoveLowCardinality(nested);
if (nested.get() == nested_no_lc.get())
return column;
return ColumnMap::create(nested_no_lc);
}
if (const auto * column_low_cardinality = typeid_cast<const ColumnLowCardinality *>(column.get()))
return column_low_cardinality->convertToFullColumn();

View File

@ -7,6 +7,7 @@
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/Serializations/SerializationMap.h>
#include <Parsers/IAST.h>
@ -53,12 +54,24 @@ DataTypeMap::DataTypeMap(const DataTypePtr & key_type_, const DataTypePtr & valu
void DataTypeMap::assertKeyType() const
{
if (!key_type->isValueRepresentedByInteger()
bool type_error = false;
if (key_type->getTypeId() == TypeIndex::LowCardinality)
{
const auto & low_cardinality_data_type = assert_cast<const DataTypeLowCardinality &>(*key_type);
if (!isStringOrFixedString(*(low_cardinality_data_type.getDictionaryType())))
type_error = true;
}
else if (!key_type->isValueRepresentedByInteger()
&& !isStringOrFixedString(*key_type)
&& !WhichDataType(key_type).isNothing()
&& !WhichDataType(key_type).isUUID())
{
type_error = true;
}
if (type_error)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Type of Map key must be a type, that can be represented by integer or string or UUID,"
"Type of Map key must be a type, that can be represented by integer or String or FixedString (possibly LowCardinality) or UUID,"
" but {} given", key_type->getName());
}

View File

@ -28,7 +28,7 @@ public:
static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet";
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override;

View File

@ -19,6 +19,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/castColumn.h>
#include <IO/WriteHelpers.h>
#include <Common/IPv6ToBinary.h>
#include <Common/formatIPv6.h>
@ -978,7 +979,8 @@ public:
!which.isDateTime64() &&
!which.isUInt() &&
!which.isFloat() &&
!which.isDecimal())
!which.isDecimal() &&
!which.isAggregateFunction())
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -990,6 +992,15 @@ public:
const IColumn * column = arguments[0].column.get();
ColumnPtr res_column;
WhichDataType which(column->getDataType());
if (which.isAggregateFunction())
{
const ColumnPtr to_string = castColumn(arguments[0], std::make_shared<DataTypeString>());
const auto * str_column = checkAndGetColumn<ColumnString>(to_string.get());
tryExecuteString(str_column, res_column);
return res_column;
}
if (tryExecuteUInt<UInt8>(column, res_column) ||
tryExecuteUInt<UInt16>(column, res_column) ||
tryExecuteUInt<UInt32>(column, res_column) ||

View File

@ -163,13 +163,6 @@ public:
arguments[0]->getName(),
getName());
if (!WhichDataType(arguments[1]).isUInt64() &&
!isTuple(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of second argument of function {} must be UInt64 or tuple(...)",
arguments[1]->getName(),
getName());
return std::make_shared<DataTypeUInt8>();
}
@ -189,8 +182,8 @@ public:
auto dictionary_key_type = dictionary->getKeyType();
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
const auto key_column = key_column_with_type.column;
const auto key_column_type = WhichDataType(key_column_with_type.type);
auto key_column = key_column_with_type.column;
auto key_column_type = key_column_with_type.type;
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
@ -214,7 +207,7 @@ public:
if (dictionary_key_type == DictionaryKeyType::simple)
{
if (!key_column_type.isUInt64())
if (!WhichDataType(key_column_type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is simple. Actual type {}.",
@ -225,24 +218,39 @@ public:
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!key_column_type.isTuple())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_column_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
key_column = key_column->convertToFullColumnIfConst();
size_t keys_size = dictionary->getStructure().getKeysSize();
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_column_with_type.type).getElements();
if (!isTuple(key_column_type))
{
if (keys_size > 1)
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be tuple when dictionary is complex and key contains more than 1 attribute."
"Actual type {}.",
getName(),
key_column_type->getName());
}
else
{
Columns tuple_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns);
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
}
}
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
return dictionary->hasKeys(key_columns, key_types);
}
else
{
if (!key_column_type.isUInt64())
if (!WhichDataType(key_column_type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is range. Actual type {}.",
@ -346,13 +354,6 @@ public:
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
auto dictionary = helper.getDictionary(dictionary_name);
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of third argument of function {}, must be UInt64 or tuple(...).",
arguments[2].type->getName(),
getName());
auto dictionary_key_type = dictionary->getKeyType();
size_t current_arguments_index = 3;
@ -446,18 +447,35 @@ public:
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!isTuple(key_col_with_type.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_col_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_col_with_type.column->convertToFullColumnIfConst();
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type;
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type))
{
if (keys_size > 1)
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be tuple when dictionary is complex and key contains more than 1 attribute."
"Actual type {}.",
getName(),
key_col_with_type.type->getName());
}
else
{
Columns tuple_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns);
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
}
}
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
result = executeDictionaryRequest(
dictionary,

View File

@ -6,6 +6,7 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Common/typeid_cast.h>
#include <Common/NaNUtils.h>
#include <Common/SipHash.h>
#include <common/range.h>
@ -40,6 +41,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
@ -304,6 +306,13 @@ void PointInPolygonWithGrid<CoordinateType>::calcGridAttributes(
y_scale = 1 / cell_height;
x_shift = -min_corner.x();
y_shift = -min_corner.y();
if (!(isFinite(x_scale)
&& isFinite(y_scale)
&& isFinite(x_shift)
&& isFinite(y_shift)
&& isFinite(grid_size)))
throw Exception("Polygon is not valid: bounding box is unbounded", ErrorCodes::BAD_ARGUMENTS);
}
template <typename CoordinateType>
@ -358,7 +367,7 @@ bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, Coordina
if (has_empty_bound)
return false;
if (std::isnan(x) || std::isnan(y))
if (!isFinite(x) || !isFinite(y))
return false;
CoordinateType float_row = (y + y_shift) * y_scale;

View File

@ -12,6 +12,7 @@
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <Common/typeid_cast.h>
@ -110,6 +111,9 @@ private:
static bool matchKeyToIndexString(const IColumn & data, const Offsets & offsets,
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
static bool matchKeyToIndexFixedString(const IColumn & data, const Offsets & offsets,
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
static bool matchKeyToIndexStringConst(const IColumn & data, const Offsets & offsets,
const Field & index, PaddedPODArray<UInt64> & matched_idxs);
@ -767,6 +771,19 @@ struct MatcherString
}
};
struct MatcherFixedString
{
const ColumnFixedString & data;
const ColumnFixedString & index;
bool match(size_t row_data, size_t row_index) const
{
auto data_ref = data.getDataAt(row_data);
auto index_ref = index.getDataAt(row_index);
return memequalSmallAllowOverflow15(index_ref.data, index_ref.size, data_ref.data, data_ref.size);
}
};
struct MatcherStringConst
{
const ColumnString & data;
@ -863,6 +880,23 @@ bool FunctionArrayElement::matchKeyToIndexString(
return true;
}
bool FunctionArrayElement::matchKeyToIndexFixedString(
const IColumn & data, const Offsets & offsets,
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs)
{
const auto * index_string = checkAndGetColumn<ColumnFixedString>(arguments[1].column.get());
if (!index_string)
return false;
const auto * data_string = checkAndGetColumn<ColumnFixedString>(&data);
if (!data_string)
return false;
MatcherFixedString matcher{*data_string, *index_string};
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
return true;
}
template <typename DataType>
bool FunctionArrayElement::matchKeyToIndexNumberConst(
const IColumn & data, const Offsets & offsets,
@ -922,8 +956,10 @@ bool FunctionArrayElement::matchKeyToIndex(
|| matchKeyToIndexNumber<Int64>(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexNumber<Int128>(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexNumber<Int256>(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexNumber<UInt256>(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexNumber<UUID>(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexString(data, offsets, arguments, matched_idxs);
|| matchKeyToIndexString(data, offsets, arguments, matched_idxs)
|| matchKeyToIndexFixedString(data, offsets, arguments, matched_idxs);
}
bool FunctionArrayElement::matchKeyToIndexConst(

View File

@ -95,6 +95,7 @@ const char * auto_contributors[] {
"Anatoly Pugachev",
"ana-uvarova",
"AnaUvarova",
"Andreas Hunkeler",
"AndreevDm",
"Andrei Bodrov",
"Andrei Chulkov",
@ -280,6 +281,7 @@ const char * auto_contributors[] {
"Dongdong Yang",
"DoomzD",
"Dr. Strange Looker",
"d.v.semenov",
"eaxdev",
"eejoin",
"egatov",
@ -290,6 +292,7 @@ const char * auto_contributors[] {
"Eldar Zaitov",
"Elena Baskakova",
"elenaspb2019",
"elevankoff",
"Elghazal Ahmed",
"Elizaveta Mironyuk",
"emakarov",
@ -434,6 +437,7 @@ const char * auto_contributors[] {
"Ivan Starkov",
"ivanzhukov",
"Ivan Zhukov",
"Jack Song",
"JackyWoo",
"Jacob Hayes",
"jakalletti",
@ -476,6 +480,7 @@ const char * auto_contributors[] {
"Konstantin Lebedev",
"Konstantin Malanchev",
"Konstantin Podshumok",
"Konstantin Rudenskii",
"Korenevskiy Denis",
"Korviakov Andrey",
"koshachy",
@ -488,6 +493,7 @@ const char * auto_contributors[] {
"kshvakov",
"kssenii",
"l",
"l1tsolaiki",
"lalex",
"Latysheva Alexandra",
"lehasm",
@ -515,6 +521,7 @@ const char * auto_contributors[] {
"long2ice",
"Lopatin Konstantin",
"Loud_Scream",
"ltybc-coder",
"luc1ph3r",
"Lucid Dreams",
"Luis Bosque",
@ -633,6 +640,7 @@ const char * auto_contributors[] {
"nicelulu",
"Nickita",
"Nickolay Yastrebov",
"nickzhwang",
"Nicolae Vartolomei",
"Nico Mandery",
"Nico Piderman",
@ -871,6 +879,7 @@ const char * auto_contributors[] {
"Veselkov Konstantin",
"vic",
"vicdashkov",
"Victor",
"Victor Tarnavsky",
"Viktor Taranenko",
"vinity",
@ -947,6 +956,7 @@ const char * auto_contributors[] {
"Yuriy Korzhenevskiy",
"Yury Karpovich",
"Yury Stankevich",
"ywill3",
"zamulla",
"zhang2014",
"zhangshengyu",
@ -957,11 +967,13 @@ const char * auto_contributors[] {
"Zhichun Wu",
"Zhipeng",
"zhukai",
"Zijie Lu",
"zlx19950903",
"Zoran Pandovski",
"zvonand",
"zvrr",
"zvvr",
"zxc111",
"zzsmdfj",
"Артем Стрельцов",
"Владислав Тихонов",
@ -980,6 +992,7 @@ const char * auto_contributors[] {
"张风啸",
"徐炘",
"曲正鹏",
"未来星___费",
"极客青年",
"谢磊",
"贾顺名(Jarvis)",

View File

@ -1 +1,6 @@
yyy
1
1
1
1
1
1

View File

@ -1,9 +1,14 @@
drop table if exists join_tbl;
DROP TABLE IF EXISTS join_tbl;
create table join_tbl (`id` String, `name` String) engine Join(any, left, id);
CREATE TABLE join_tbl (`id` String, `name` String, lcname LowCardinality(String)) ENGINE = Join(any, left, id);
insert into join_tbl values ('xxx', 'yyy');
INSERT INTO join_tbl VALUES ('xxx', 'yyy', 'yyy');
select joinGet('join_tbl', 'name', toLowCardinality('xxx'));
SELECT joinGet('join_tbl', 'name', 'xxx') == 'yyy';
SELECT joinGet('join_tbl', 'name', toLowCardinality('xxx')) == 'yyy';
SELECT joinGet('join_tbl', 'name', toLowCardinality(materialize('xxx'))) == 'yyy';
SELECT joinGet('join_tbl', 'lcname', 'xxx') == 'yyy';
SELECT joinGet('join_tbl', 'lcname', toLowCardinality('xxx')) == 'yyy';
SELECT joinGet('join_tbl', 'lcname', toLowCardinality(materialize('xxx'))) == 'yyy';
drop table if exists join_tbl;
DROP TABLE IF EXISTS join_tbl;

View File

@ -0,0 +1,2 @@
b
{'1':1} 1 0

View File

@ -0,0 +1,12 @@
DROP TABLE IF EXISTS map_lc;
SET allow_experimental_map_type = 1;
CREATE TABLE map_lc
(
`kv` Map(LowCardinality(String), LowCardinality(String))
)
ENGINE = Memory;
INSERT INTO map_lc select map('a', 'b');
SELECT kv['a'] FROM map_lc;
DROP TABLE map_lc;
SELECT map(toFixedString('1',1),1) AS m, m[toFixedString('1',1)],m[toFixedString('1',2)];

View File

@ -33,3 +33,7 @@
1
1
1
1
1
2D000000000000000A
001011010000000000000000000000000000000000000000000000000000000000001010

View File

@ -37,3 +37,9 @@ select bin(unbin('0')) == '00000000';
select hex('') == bin('');
select unhex('') == unbin('');
select unhex('0') == unbin('0');
-- hex and bin support AggregateFunction
select hex(sumState(number)) == hex(toString(sumState(number))) from numbers(10);
select hex(avgState(number)) == hex(toString(avgState(number))) from numbers(99);
select hex(avgState(number)) from numbers(10);
select bin(avgState(number)) from numbers(10);

View File

@ -0,0 +1,2 @@
SET validate_polygons = 0;
SELECT pointInPolygon((-inf, 1023), [(10.000100135803223, 10000000000.), (inf, 0.9998999834060669), (1.1920928955078125e-7, 100.0000991821289), (1.000100016593933, 100.0000991821289)]);

View File

@ -0,0 +1,10 @@
dictGet
Value
Value
Value
Value
dictHas
1
1
1
1

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS test_dictionary_source;
CREATE TABLE test_dictionary_source (key String, value String) ENGINE=TinyLog;
INSERT INTO test_dictionary_source VALUES ('Key', 'Value');
DROP DICTIONARY IF EXISTS test_dictionary;
CREATE DICTIONARY test_dictionary(key String, value String)
PRIMARY KEY key
LAYOUT(COMPLEX_KEY_HASHED())
SOURCE(CLICKHOUSE(TABLE 'test_dictionary_source'))
LIFETIME(0);
SELECT 'dictGet';
SELECT dictGet('test_dictionary', 'value', tuple('Key'));
SELECT dictGet('test_dictionary', 'value', tuple(materialize('Key')));
SELECT dictGet('test_dictionary', 'value', 'Key');
SELECT dictGet('test_dictionary', 'value', materialize('Key'));
SELECT 'dictHas';
SELECT dictHas('test_dictionary', tuple('Key'));
SELECT dictHas('test_dictionary', tuple(materialize('Key')));
SELECT dictHas('test_dictionary', 'Key');
SELECT dictHas('test_dictionary', materialize('Key'));
DROP DICTIONARY test_dictionary;
DROP TABLE test_dictionary_source;

View File

@ -1,3 +1,4 @@
v21.7.2.7-stable 2021-07-09
v21.6.6.51-stable 2021-07-02
v21.6.5.37-stable 2021-06-19
v21.6.4.26-stable 2021-06-11

1 v21.6.6.51-stable v21.7.2.7-stable 2021-07-02 2021-07-09
1 v21.7.2.7-stable 2021-07-09
2 v21.6.6.51-stable v21.6.6.51-stable 2021-07-02 2021-07-02
3 v21.6.5.37-stable v21.6.5.37-stable 2021-06-19 2021-06-19
4 v21.6.4.26-stable v21.6.4.26-stable 2021-06-11 2021-06-11