Merge branch 'master' into analyzer-execution-names

This commit is contained in:
Nikolai Kochetov 2023-12-13 17:21:26 +01:00 committed by GitHub
commit c05c232974
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 835 additions and 259 deletions

View File

@ -543,26 +543,52 @@ Like `concatWithSeparator` but assumes that `concatWithSeparator(sep, expr1, exp
A function is called injective if it returns for different arguments different results. In other words: different arguments never produce identical result.
## substring(s, offset, length)
## substring
Returns a substring with `length` many bytes, starting at the byte at index `offset`. Character indexing starts from 1.
Returns the substring of a string `s` which starts at the specified byte index `offset`. Byte counting starts from 1. If `offset` is 0, an empty string is returned. If `offset` is negative, the substring starts `pos` characters from the end of the string, rather than from the beginning. An optional argument `length` specifies the maximum number of bytes the returned substring may have.
**Syntax**
```sql
substring(s, offset, length)
substring(s, offset[, length])
```
Alias:
- `substr`
- `mid`
**Arguments**
- `s` — The string to calculate a substring from. [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md) or [Enum](../../sql-reference/data-types/enum.md)
- `offset` — The starting position of the substring in `s` . [(U)Int*](../../sql-reference/data-types/int-uint.md).
- `length` — The maximum length of the substring. [(U)Int*](../../sql-reference/data-types/int-uint.md). Optional.
**Returned value**
A substring of `s` with `length` many bytes, starting at index `offset`.
Type: `String`.
**Example**
``` sql
SELECT 'database' AS db, substr(db, 5), substr(db, 5, 1)
```
Result:
```result
┌─db───────┬─substring('database', 5)─┬─substring('database', 5, 1)─┐
│ database │ base │ b │
└──────────┴──────────────────────────┴─────────────────────────────┘
```
## substringUTF8
Like `substring` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.
## substringIndex(s, delim, count)
## substringIndex
Returns the substring of `s` before `count` occurrences of the delimiter `delim`, as in Spark or MySQL.
@ -593,7 +619,7 @@ Result:
└──────────────────────────────────────────────┘
```
## substringIndexUTF8(s, delim, count)
## substringIndexUTF8
Like `substringIndex` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.

View File

@ -68,6 +68,7 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ServerType.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperReadinessHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnection.cpp

View File

@ -33,6 +33,7 @@
#include <Server/HTTP/HTTPServer.h>
#include <Server/TCPServer.h>
#include <Server/HTTPHandlerFactory.h>
#include <Server/KeeperReadinessHandler.h>
#include "Core/Defines.h"
#include "config.h"
@ -494,6 +495,29 @@ try
std::make_unique<HTTPServer>(
std::move(my_http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
});
/// HTTP control endpoints
port_name = "keeper_server.http_control.port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port) mutable
{
auto my_http_context = httpContext();
Poco::Timespan my_keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
Poco::Net::HTTPServerParams::Ptr my_http_params = new Poco::Net::HTTPServerParams;
my_http_params->setTimeout(my_http_context->getReceiveTimeout());
my_http_params->setKeepAliveTimeout(my_keep_alive_timeout);
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(my_http_context->getReceiveTimeout());
socket.setSendTimeout(my_http_context->getSendTimeout());
servers->emplace_back(
listen_host,
port_name,
"HTTP Control: http://" + address.toString(),
std::make_unique<HTTPServer>(
std::move(my_http_context), createKeeperHTTPControlMainHandlerFactory(config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)
);
});
}
for (auto & server : *servers)

View File

@ -92,6 +92,7 @@
#include <Server/ProxyV1HandlerFactory.h>
#include <Server/TLSHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/KeeperReadinessHandler.h>
#include <Server/HTTP/HTTPServer.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Core/ServerSettings.h>
@ -1554,6 +1555,34 @@ try
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
});
/// HTTP control endpoints
port_name = "keeper_server.http_control.port";
createServer(config(), listen_host, port_name, listen_try, /* start_server: */ false,
servers_to_start_before_tables,
[&](UInt16 port) -> ProtocolServerAdapter
{
auto http_context = httpContext();
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(http_context->getReceiveTimeout());
http_params->setKeepAliveTimeout(keep_alive_timeout);
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config(), socket, listen_host, port);
socket.setReceiveTimeout(http_context->getReceiveTimeout());
socket.setSendTimeout(http_context->getSendTimeout());
return ProtocolServerAdapter(
listen_host,
port_name,
"HTTP Control: http://" + address.toString(),
std::make_unique<HTTPServer>(
std::move(http_context),
createKeeperHTTPControlMainHandlerFactory(
config_getter(),
global_context->getKeeperDispatcher(),
"KeeperHTTPControlHandler-factory"), server_pool, socket, http_params));
});
}
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination.");

View File

@ -2154,34 +2154,20 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
node_to_replace = &sort_node->getExpression();
auto * constant_node = (*node_to_replace)->as<ConstantNode>();
if (!constant_node
|| (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64))
if (!constant_node || constant_node->getValue().getType() != Field::Types::UInt64)
continue;
UInt64 pos;
if (constant_node->getValue().getType() == Field::Types::UInt64)
{
pos = constant_node->getValue().get<UInt64>();
}
else // Int64
{
auto value = constant_node->getValue().get<Int64>();
pos = value > 0 ? value : projection_nodes.size() + value + 1;
}
if (!pos || pos > projection_nodes.size())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
UInt64 positional_argument_number = constant_node->getValue().get<UInt64>();
if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}",
pos,
positional_argument_number,
projection_nodes.size(),
scope.scope_node->formatASTForErrorMessage());
--pos;
*node_to_replace = projection_nodes[pos]->clone();
if (auto it = resolved_expressions.find(projection_nodes[pos]);
--positional_argument_number;
*node_to_replace = projection_nodes[positional_argument_number]->clone();
if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]);
it != resolved_expressions.end())
{
resolved_expressions[*node_to_replace] = it->second;

View File

@ -440,6 +440,8 @@ template <typename T> inline bool isFloat(const T & data_type) { return WhichDat
template <typename T> inline bool isNativeNumber(const T & data_type) { return WhichDataType(data_type).isNativeNumber(); }
template <typename T> inline bool isNumber(const T & data_type) { return WhichDataType(data_type).isNumber(); }
template <typename T> inline bool isEnum8(const T & data_type) { return WhichDataType(data_type).isEnum8(); }
template <typename T> inline bool isEnum16(const T & data_type) { return WhichDataType(data_type).isEnum16(); }
template <typename T> inline bool isEnum(const T & data_type) { return WhichDataType(data_type).isEnum(); }
template <typename T> inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }

View File

@ -11,6 +11,8 @@
#include <Common/typeid_cast.h>
#include <Common/UTF8Helpers.h>
#include <DataTypes/EnumValues.h>
#include "IArraySource.h"
#include "IValueSource.h"
#include "Slices.h"
@ -56,8 +58,8 @@ struct NumericArraySource : public ArraySourceImpl<NumericArraySource<T>>
}
explicit NumericArraySource(const ColumnArray & arr)
: column(typeid_cast<const ColVecType &>(arr.getData()))
, elements(typeid_cast<const ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
: column(typeid_cast<const ColVecType &>(arr.getData()))
, elements(typeid_cast<const ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
{
}
@ -154,17 +156,22 @@ struct ConstSource : public Base
size_t row_num = 0;
explicit ConstSource(const ColumnConst & col_)
: Base(static_cast<const typename Base::Column &>(col_.getDataColumn())), total_rows(col_.size())
: Base(static_cast<const typename Base::Column &>(col_.getDataColumn()))
, total_rows(col_.size())
{
}
template <typename ColumnType>
ConstSource(const ColumnType & col_, size_t total_rows_) : Base(col_), total_rows(total_rows_)
ConstSource(const ColumnType & col_, size_t total_rows_)
: Base(col_)
, total_rows(total_rows_)
{
}
template <typename ColumnType>
ConstSource(const ColumnType & col_, const NullMap & null_map_, size_t total_rows_) : Base(col_, null_map_), total_rows(total_rows_)
ConstSource(const ColumnType & col_, const NullMap & null_map_, size_t total_rows_)
: Base(col_, null_map_)
, total_rows(total_rows_)
{
}
@ -240,7 +247,8 @@ struct StringSource
ColumnString::Offset prev_offset = 0;
explicit StringSource(const ColumnString & col)
: elements(col.getChars()), offsets(col.getOffsets())
: elements(col.getChars())
, offsets(col.getOffsets())
{
}
@ -313,6 +321,96 @@ struct StringSource
}
};
/// Treats Enum values as Strings, modeled after StringSource
template <typename EnumDataType>
struct EnumSource
{
using Column = typename EnumDataType::ColumnType;
using Slice = NumericArraySlice<UInt8>;
using SinkType = StringSink;
const typename Column::Container & data;
const EnumDataType & data_type;
size_t row_num = 0;
EnumSource(const Column & col, const EnumDataType & data_type_)
: data(col.getData())
, data_type(data_type_)
{
}
void next()
{
++row_num;
}
bool isEnd() const
{
return row_num == data.size();
}
size_t rowNum() const
{
return row_num;
}
size_t getSizeForReserve() const
{
return data.size();
}
size_t getElementSize() const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
return name.size();
}
size_t getColumnSize() const
{
return data.size();
}
Slice getWhole() const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
return {reinterpret_cast<const UInt8 *>(name.data()), name.size()};
}
Slice getSliceFromLeft(size_t offset) const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
if (offset >= name.size())
return {reinterpret_cast<const UInt8 *>(name.data()), 0};
return {reinterpret_cast<const UInt8 *>(name.data()) + offset, name.size() - offset};
}
Slice getSliceFromLeft(size_t offset, size_t length) const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
if (offset >= name.size())
return {reinterpret_cast<const UInt8 *>(name.data()), 0};
return {reinterpret_cast<const UInt8 *>(name.data()) + offset, std::min(length, name.size() - offset)};
}
Slice getSliceFromRight(size_t offset) const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
if (offset > name.size())
return {reinterpret_cast<const UInt8 *>(name.data()), name.size()};
return {reinterpret_cast<const UInt8 *>(name.data()) + name.size() - offset, offset};
}
Slice getSliceFromRight(size_t offset, size_t length) const
{
std::string_view name = data_type.getNameForValue(data[row_num]).toView();
if (offset > name.size())
return {reinterpret_cast<const UInt8 *>(name.data()), length + name.size() > offset ? std::min(name.size(), length + name.size() - offset) : 0};
return {reinterpret_cast<const UInt8 *>(name.data()) + name.size() - offset, std::min(length, offset)};
}
};
/// Differs to StringSource by having 'offset' and 'length' in code points instead of bytes in getSlice* methods.
/** NOTE: The behaviour of substring and substringUTF8 is inconsistent when negative offset is greater than string size:
@ -419,7 +517,7 @@ struct FixedStringSource
size_t column_size = 0;
explicit FixedStringSource(const ColumnFixedString & col)
: string_size(col.getN())
: string_size(col.getN())
{
const auto & chars = col.getChars();
pos = chars.data();
@ -553,7 +651,8 @@ struct GenericArraySource : public ArraySourceImpl<GenericArraySource>
}
explicit GenericArraySource(const ColumnArray & arr)
: elements(arr.getData()), offsets(arr.getOffsets())
: elements(arr.getData())
, offsets(arr.getOffsets())
{
}
@ -813,7 +912,10 @@ struct NullableValueSource : public ValueSource
const NullMap & null_map;
template <typename Column>
explicit NullableValueSource(const Column & col, const NullMap & null_map_) : ValueSource(col), null_map(null_map_) {}
NullableValueSource(const Column & col, const NullMap & null_map_)
: ValueSource(col)
, null_map(null_map_)
{}
void accept(ValueSourceVisitor & visitor) override { visitor.visit(*this); }

View File

@ -1,15 +1,16 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Functions/GatherUtils/Algorithms.h>
#include <Functions/GatherUtils/GatherUtils.h>
#include <Functions/GatherUtils/Sources.h>
#include <Functions/GatherUtils/Sinks.h>
#include <Functions/GatherUtils/Slices.h>
#include <Functions/GatherUtils/Algorithms.h>
#include <Functions/GatherUtils/Sources.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
@ -20,50 +21,50 @@ using namespace GatherUtils;
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ZERO_ARRAY_OR_TUPLE_INDEX;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ZERO_ARRAY_OR_TUPLE_INDEX;
}
namespace
{
/// If 'is_utf8' - measure offset and length in code points instead of bytes.
/// UTF8 variant is not available for FixedString arguments.
template <bool is_utf8>
class FunctionSubstring : public IFunction
{
public:
static constexpr auto name = is_utf8 ? "substringUTF8" : "substring";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionSubstring>();
}
String getName() const override
{
return name;
}
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionSubstring>(); }
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
const size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: "
"passed {}, should be 2 or 3", getName(), number_of_arguments);
if ((is_utf8 && !isString(arguments[0])) || !isStringOrFixedString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
arguments[0]->getName(), getName());
if constexpr (is_utf8)
{
/// UTF8 variant is not available for FixedString and Enum arguments.
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}",
arguments[0]->getName(), getName());
}
else
{
if (!isStringOrFixedString(arguments[0]) && !isEnum(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}",
arguments[0]->getName(), getName());
}
if (!isNativeNumber(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}",
@ -77,44 +78,40 @@ public:
}
template <typename Source>
ColumnPtr executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length,
const ColumnConst * column_start_const, const ColumnConst * column_length_const,
Int64 start_value, Int64 length_value, Source && source,
size_t input_rows_count) const
ColumnPtr executeForSource(const ColumnPtr & column_offset, const ColumnPtr & column_length,
bool column_offset_const, bool column_length_const,
Int64 offset, Int64 length,
Source && source, size_t input_rows_count) const
{
auto col_res = ColumnString::create();
if (!column_length)
{
if (column_start_const)
if (column_offset_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetUnbounded(
source, StringSink(*col_res, input_rows_count), static_cast<size_t>(start_value - 1));
else if (start_value < 0)
sliceFromRightConstantOffsetUnbounded(
source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value));
if (offset > 0)
sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), static_cast<size_t>(offset - 1));
else if (offset < 0)
sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(offset));
else
throw Exception(ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX, "Indices in strings are 1-based");
}
else
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start);
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_offset);
}
else
{
if (column_start_const && column_length_const)
if (column_offset_const && column_length_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetBounded(
source, StringSink(*col_res, input_rows_count), static_cast<size_t>(start_value - 1), length_value);
else if (start_value < 0)
sliceFromRightConstantOffsetBounded(
source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value), length_value);
if (offset > 0)
sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), static_cast<size_t>(offset - 1), length);
else if (offset < 0)
sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(offset), length);
else
throw Exception(ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX, "Indices in strings are 1-based");
}
else
sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length);
sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_offset, *column_length);
}
return col_res;
@ -122,58 +119,60 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
size_t number_of_arguments = arguments.size();
const size_t number_of_arguments = arguments.size();
ColumnPtr column_string = arguments[0].column;
ColumnPtr column_start = arguments[1].column;
ColumnPtr column_offset = arguments[1].column;
ColumnPtr column_length;
if (number_of_arguments == 3)
column_length = arguments[2].column;
const ColumnConst * column_start_const = checkAndGetColumn<ColumnConst>(column_start.get());
const ColumnConst * column_offset_const = checkAndGetColumn<ColumnConst>(column_offset.get());
const ColumnConst * column_length_const = nullptr;
if (number_of_arguments == 3)
column_length_const = checkAndGetColumn<ColumnConst>(column_length.get());
Int64 start_value = 0;
Int64 length_value = 0;
Int64 offset = 0;
Int64 length = 0;
if (column_start_const)
start_value = column_start_const->getInt(0);
if (column_offset_const)
offset = column_offset_const->getInt(0);
if (column_length_const)
length_value = column_length_const->getInt(0);
length = column_length_const->getInt(0);
if constexpr (is_utf8)
{
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, UTF8StringSource(*col), input_rows_count);
else if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
arguments[0].column->getName(), getName());
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count);
if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
}
else
{
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, StringSource(*col), input_rows_count);
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, FixedStringSource(*col_fixed), input_rows_count);
else if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, ConstSource<StringSource>(*col_const), input_rows_count);
else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, ConstSource<FixedStringSource>(*col_const_fixed), input_rows_count);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
arguments[0].column->getName(), getName());
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count);
if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, FixedStringSource(*col_fixed), input_rows_count);
if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<StringSource>(*col_const), input_rows_count);
if (const ColumnConst * col_const_fixed = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<FixedStringSource>(*col_const_fixed), input_rows_count);
if (isEnum(arguments[0].type))
{
if (const typename DataTypeEnum8::ColumnType * col_enum8 = checkAndGetColumn<typename DataTypeEnum8::ColumnType>(column_string.get()))
{
const auto * type_enum8 = assert_cast<const DataTypeEnum8 *>(arguments[0].type.get());
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, EnumSource<DataTypeEnum8>(*col_enum8, *type_enum8), input_rows_count);
}
if (const typename DataTypeEnum16::ColumnType * col_enum16 = checkAndGetColumn<typename DataTypeEnum16::ColumnType>(column_string.get()))
{
const auto * type_enum16 = assert_cast<const DataTypeEnum16 *>(arguments[0].type.get());
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, EnumSource<DataTypeEnum16>(*col_enum16, *type_enum16), input_rows_count);
}
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
}
}
};
@ -183,8 +182,8 @@ public:
REGISTER_FUNCTION(Substring)
{
factory.registerFunction<FunctionSubstring<false>>({}, FunctionFactory::CaseInsensitive);
factory.registerAlias("substr", "substring", FunctionFactory::CaseInsensitive);
factory.registerAlias("mid", "substring", FunctionFactory::CaseInsensitive); /// from MySQL dialect
factory.registerAlias("substr", "substring", FunctionFactory::CaseInsensitive); // MySQL alias
factory.registerAlias("mid", "substring", FunctionFactory::CaseInsensitive); /// MySQL alias
factory.registerFunction<FunctionSubstring<true>>({}, FunctionFactory::CaseSensitive);
}

View File

@ -27,29 +27,14 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel
return false;
auto which = ast_literal->value.getType();
if (which != Field::Types::UInt64 && which != Field::Types::Int64)
if (which != Field::Types::UInt64)
return false;
UInt64 pos;
if (which == Field::Types::UInt64)
{
pos = ast_literal->value.get<UInt64>();
}
else if (which == Field::Types::Int64)
{
auto value = ast_literal->value.get<Int64>();
pos = value > 0 ? value : columns.size() + value + 1;
}
else
{
return false;
}
auto pos = ast_literal->value.get<UInt64>();
if (!pos || pos > columns.size())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Positional argument out of bounds: {} (expected in range [1, {}]",
pos, columns.size());
const auto & column = columns[--pos];
if (typeid_cast<const ASTIdentifier *>(column.get()) || typeid_cast<const ASTLiteral *>(column.get()))

View File

@ -0,0 +1,94 @@
#include <Server/KeeperReadinessHandler.h>
#if USE_NURAFT
#include <memory>
#include <IO/HTTPCommon.h>
#include <Coordination/KeeperDispatcher.h>
#include <Server/HTTPHandlerFactory.h>
#include <Server/HTTPHandlerRequestFilter.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
namespace DB
{
void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTPServerResponse & response)
{
try
{
auto is_leader = keeper_dispatcher->isLeader();
auto is_follower = keeper_dispatcher->isFollower() && keeper_dispatcher->hasLeader();
auto is_observer = keeper_dispatcher->isObserver() && keeper_dispatcher->hasLeader();
auto data = keeper_dispatcher->getKeeper4LWInfo();
auto status = is_leader || is_follower || is_observer;
Poco::JSON::Object json, details;
details.set("role", data.getRole());
details.set("hasLeader", keeper_dispatcher->hasLeader());
json.set("details", details);
json.set("status", status ? "ok" : "fail");
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
oss.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(json, oss);
if (!status)
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE);
*response.send() << oss.str();
}
catch (...)
{
tryLogCurrentException("KeeperReadinessHandler");
try
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
if (!response.sent())
{
/// We have not sent anything yet and we don't even know if we need to compress response.
*response.send() << getCurrentExceptionMessage(false) << std::endl;
}
}
catch (...)
{
LOG_ERROR((&Poco::Logger::get("KeeperReadinessHandler")), "Cannot send exception to client");
}
}
}
HTTPRequestHandlerFactoryPtr createKeeperHTTPControlMainHandlerFactory(
const Poco::Util::AbstractConfiguration & config,
std::shared_ptr<KeeperDispatcher> keeper_dispatcher,
const std::string & name)
{
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
using Factory = HandlingRuleHTTPHandlerFactory<KeeperReadinessHandler>;
Factory::Creator creator = [keeper_dispatcher]() -> std::unique_ptr<KeeperReadinessHandler>
{
return std::make_unique<KeeperReadinessHandler>(keeper_dispatcher);
};
auto readiness_handler = std::make_shared<Factory>(std::move(creator));
readiness_handler->attachStrictPath(config.getString("keeper_server.http_control.readiness.endpoint", "/ready"));
readiness_handler->allowGetAndHeadRequest();
factory->addHandler(readiness_handler);
return factory;
}
}
#endif

View File

@ -0,0 +1,36 @@
#pragma once
#include <config.h>
#if USE_NURAFT
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Coordination/KeeperDispatcher.h>
namespace DB
{
class KeeperReadinessHandler : public HTTPRequestHandler, WithContext
{
private:
std::shared_ptr<KeeperDispatcher> keeper_dispatcher;
public:
explicit KeeperReadinessHandler(std::shared_ptr<KeeperDispatcher> keeper_dispatcher_)
: keeper_dispatcher(keeper_dispatcher_)
{
}
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
};
HTTPRequestHandlerFactoryPtr
createKeeperHTTPControlMainHandlerFactory(
const Poco::Util::AbstractConfiguration & config,
std::shared_ptr<KeeperDispatcher> keeper_dispatcher,
const std::string & name);
}
#endif

View File

@ -279,6 +279,13 @@ def get_leader(cluster, nodes):
raise Exception("No leader in Keeper cluster.")
def get_any_follower(cluster, nodes):
for node in nodes:
if is_follower(cluster, node):
return node
raise Exception("No followers in Keeper cluster.")
def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient:
_fake = KazooClient(
hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout

View File

@ -0,0 +1,37 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
<http_control>
<port>9182</port>
</http_control>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,37 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
<http_control>
<port>9182</port>
</http_control>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,37 @@
<clickhouse>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>9234</port>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>9234</port>
<start_as_follower>true</start_as_follower>
</server>
</raft_configuration>
<http_control>
<port>9182</port>
</http_control>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,74 @@
#!/usr/bin/env python3
import os
import pytest
import requests
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
import helpers.keeper_utils as keeper_utils
cluster = ClickHouseCluster(__file__)
CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs")
node1 = cluster.add_instance(
"node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True
)
node3 = cluster.add_instance(
"node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_http_readiness_basic_responses(started_cluster):
leader = keeper_utils.get_leader(cluster, [node1, node2, node3])
response = requests.get(
"http://{host}:{port}/ready".format(host=leader.ip_address, port=9182)
)
assert response.status_code == 200
readiness_data = response.json()
assert readiness_data["status"] == "ok"
assert readiness_data["details"]["role"] == "leader"
follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3])
response = requests.get(
"http://{host}:{port}/ready".format(host=follower.ip_address, port=9182)
)
assert response.status_code == 200
readiness_data = response.json()
assert readiness_data["status"] == "ok"
assert readiness_data["details"]["role"] == "follower"
assert readiness_data["details"]["hasLeader"] == True
def test_http_readiness_partitioned_cluster(started_cluster):
with PartitionManager() as pm:
leader = keeper_utils.get_leader(cluster, [node1, node2, node3])
follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3])
pm.partition_instances(leader, follower)
keeper_utils.wait_until_quorum_lost(cluster, follower)
response = requests.get(
"http://{host}:{port}/ready".format(host=follower.ip_address, port=9182)
)
print(response.json())
assert response.status_code == 503
readiness_data = response.json()
assert readiness_data["status"] == "fail"
assert readiness_data["details"]["role"] == "follower"
assert readiness_data["details"]["hasLeader"] == False

View File

@ -129,6 +129,9 @@ def test_parallel_replicas_over_distributed(
node = nodes[0]
expected_result = f"6003\t-1999\t1999\t3\n"
# sync all replicas to get consistent result
node.query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster} {table_name}")
# parallel replicas
assert (
node.query(
@ -142,11 +145,13 @@ def test_parallel_replicas_over_distributed(
== expected_result
)
# sync all replicas to get consistent result by next distributed query
node.query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster} {table_name}")
# w/o parallel replicas
assert (
node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d")
node.query(
f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d",
settings={
"allow_experimental_parallel_reading_from_replicas": 0,
},
)
== expected_result
)

View File

@ -0,0 +1,155 @@
-- Positive offsets (slice from left)
Offset: 1 Length: 1 hello shark h s
Offset: 1 Length: 2 hello shark he sh
Offset: 1 Length: 3 hello shark hel sha
Offset: 1 Length: 4 hello shark hell shar
Offset: 1 Length: 5 hello shark hello shark
Offset: 1 Length: 6 hello shark hello shark
Offset: 2 Length: 1 ello hark e h
Offset: 2 Length: 2 ello hark el ha
Offset: 2 Length: 3 ello hark ell har
Offset: 2 Length: 4 ello hark ello hark
Offset: 2 Length: 5 ello hark ello hark
Offset: 2 Length: 6 ello hark ello hark
Offset: 3 Length: 1 llo ark l a
Offset: 3 Length: 2 llo ark ll ar
Offset: 3 Length: 3 llo ark llo ark
Offset: 3 Length: 4 llo ark llo ark
Offset: 3 Length: 5 llo ark llo ark
Offset: 3 Length: 6 llo ark llo ark
Offset: 4 Length: 1 lo rk l r
Offset: 4 Length: 2 lo rk lo rk
Offset: 4 Length: 3 lo rk lo rk
Offset: 4 Length: 4 lo rk lo rk
Offset: 4 Length: 5 lo rk lo rk
Offset: 4 Length: 6 lo rk lo rk
Offset: 5 Length: 1 o k o k
Offset: 5 Length: 2 o k o k
Offset: 5 Length: 3 o k o k
Offset: 5 Length: 4 o k o k
Offset: 5 Length: 5 o k o k
Offset: 5 Length: 6 o k o k
Offset: 6 Length: 1
Offset: 6 Length: 2
Offset: 6 Length: 3
Offset: 6 Length: 4
Offset: 6 Length: 5
Offset: 6 Length: 6
Offset: 1 Length: 1 world eagle w e
Offset: 1 Length: 2 world eagle wo ea
Offset: 1 Length: 3 world eagle wor eag
Offset: 1 Length: 4 world eagle worl eagl
Offset: 1 Length: 5 world eagle world eagle
Offset: 1 Length: 6 world eagle world eagle
Offset: 2 Length: 1 orld agle o a
Offset: 2 Length: 2 orld agle or ag
Offset: 2 Length: 3 orld agle orl agl
Offset: 2 Length: 4 orld agle orld agle
Offset: 2 Length: 5 orld agle orld agle
Offset: 2 Length: 6 orld agle orld agle
Offset: 3 Length: 1 rld gle r g
Offset: 3 Length: 2 rld gle rl gl
Offset: 3 Length: 3 rld gle rld gle
Offset: 3 Length: 4 rld gle rld gle
Offset: 3 Length: 5 rld gle rld gle
Offset: 3 Length: 6 rld gle rld gle
Offset: 4 Length: 1 ld le l l
Offset: 4 Length: 2 ld le ld le
Offset: 4 Length: 3 ld le ld le
Offset: 4 Length: 4 ld le ld le
Offset: 4 Length: 5 ld le ld le
Offset: 4 Length: 6 ld le ld le
Offset: 5 Length: 1 d e d e
Offset: 5 Length: 2 d e d e
Offset: 5 Length: 3 d e d e
Offset: 5 Length: 4 d e d e
Offset: 5 Length: 5 d e d e
Offset: 5 Length: 6 d e d e
Offset: 6 Length: 1
Offset: 6 Length: 2
Offset: 6 Length: 3
Offset: 6 Length: 4
Offset: 6 Length: 5
Offset: 6 Length: 6
-- Negative offsets (slice from right)
Offset: -1 Length: 1 o k o k
Offset: -1 Length: 2 o k o k
Offset: -1 Length: 3 o k o k
Offset: -1 Length: 4 o k o k
Offset: -1 Length: 5 o k o k
Offset: -1 Length: 6 o k o k
Offset: -2 Length: 1 lo rk l r
Offset: -2 Length: 2 lo rk lo rk
Offset: -2 Length: 3 lo rk lo rk
Offset: -2 Length: 4 lo rk lo rk
Offset: -2 Length: 5 lo rk lo rk
Offset: -2 Length: 6 lo rk lo rk
Offset: -3 Length: 1 llo ark l a
Offset: -3 Length: 2 llo ark ll ar
Offset: -3 Length: 3 llo ark llo ark
Offset: -3 Length: 4 llo ark llo ark
Offset: -3 Length: 5 llo ark llo ark
Offset: -3 Length: 6 llo ark llo ark
Offset: -4 Length: 1 ello hark e h
Offset: -4 Length: 2 ello hark el ha
Offset: -4 Length: 3 ello hark ell har
Offset: -4 Length: 4 ello hark ello hark
Offset: -4 Length: 5 ello hark ello hark
Offset: -4 Length: 6 ello hark ello hark
Offset: -5 Length: 1 hello shark h s
Offset: -5 Length: 2 hello shark he sh
Offset: -5 Length: 3 hello shark hel sha
Offset: -5 Length: 4 hello shark hell shar
Offset: -5 Length: 5 hello shark hello shark
Offset: -5 Length: 6 hello shark hello shark
Offset: -6 Length: 1 hello shark
Offset: -6 Length: 2 hello shark h s
Offset: -6 Length: 3 hello shark he sh
Offset: -6 Length: 4 hello shark hel sha
Offset: -6 Length: 5 hello shark hell shar
Offset: -6 Length: 6 hello shark hello shark
Offset: -1 Length: 1 d e d e
Offset: -1 Length: 2 d e d e
Offset: -1 Length: 3 d e d e
Offset: -1 Length: 4 d e d e
Offset: -1 Length: 5 d e d e
Offset: -1 Length: 6 d e d e
Offset: -2 Length: 1 ld le l l
Offset: -2 Length: 2 ld le ld le
Offset: -2 Length: 3 ld le ld le
Offset: -2 Length: 4 ld le ld le
Offset: -2 Length: 5 ld le ld le
Offset: -2 Length: 6 ld le ld le
Offset: -3 Length: 1 rld gle r g
Offset: -3 Length: 2 rld gle rl gl
Offset: -3 Length: 3 rld gle rld gle
Offset: -3 Length: 4 rld gle rld gle
Offset: -3 Length: 5 rld gle rld gle
Offset: -3 Length: 6 rld gle rld gle
Offset: -4 Length: 1 orld agle o a
Offset: -4 Length: 2 orld agle or ag
Offset: -4 Length: 3 orld agle orl agl
Offset: -4 Length: 4 orld agle orld agle
Offset: -4 Length: 5 orld agle orld agle
Offset: -4 Length: 6 orld agle orld agle
Offset: -5 Length: 1 world eagle w e
Offset: -5 Length: 2 world eagle wo ea
Offset: -5 Length: 3 world eagle wor eag
Offset: -5 Length: 4 world eagle worl eagl
Offset: -5 Length: 5 world eagle world eagle
Offset: -5 Length: 6 world eagle world eagle
Offset: -6 Length: 1 world eagle
Offset: -6 Length: 2 world eagle w e
Offset: -6 Length: 3 world eagle wo ea
Offset: -6 Length: 4 world eagle wor eag
Offset: -6 Length: 5 world eagle worl eagl
Offset: -6 Length: 6 world eagle world eagle
-- Zero offset/length
Offset: 0 Length: 0
Offset: 0 Length: 1
Offset: 1 Length: 0 hello shark
Offset: 0 Length: 0
Offset: 0 Length: 1
Offset: 1 Length: 0 world eagle
-- Constant enums
f fo

View File

@ -0,0 +1,32 @@
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(e8 Enum8('hello' = -5, 'world' = 15), e16 Enum16('shark' = -999, 'eagle' = 9999)) ENGINE MergeTree ORDER BY tuple();
INSERT INTO TABLE tab VALUES ('hello', 'shark'), ('world', 'eagle');
SELECT '-- Positive offsets (slice from left)';
WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6),
permutations AS (SELECT c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2)
SELECT 'Offset: ', p.offset, 'Length: ', p.length,
substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2,
substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4
FROM tab LEFT JOIN permutations AS p ON true;
SELECT '-- Negative offsets (slice from right)';
WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6),
permutations AS (SELECT -c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2)
SELECT 'Offset: ', p.offset, 'Length: ', p.length,
substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2,
substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4
FROM tab LEFT JOIN permutations AS p ON true;
SELECT '-- Zero offset/length';
WITH cte AS (SELECT number AS n FROM system.numbers LIMIT 2),
permutations AS (SELECT c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2 LIMIT 3)
SELECT 'Offset: ', p.offset, 'Length: ', p.length,
substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2,
substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4
FROM tab LEFT JOIN permutations AS p ON true;
SELECT '-- Constant enums';
SELECT substring(CAST('foo', 'Enum8(\'foo\' = 1)'), 1, 1), substring(CAST('foo', 'Enum16(\'foo\' = 1111)'), 1, 2);
DROP TABLE tab;

View File

@ -28,7 +28,7 @@ do
$CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" 2>&1| grep -Ev "Removing leftovers from table|removed by another replica"
$CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test"
if [[ $engine == *"ReplicatedMergeTree"* ]]; then
$CLICKHOUSE_CLIENT --enable_positional_arguments=0 -q "ALTER TABLE test
$CLICKHOUSE_CLIENT -q "ALTER TABLE test
UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 'dummy')[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic"
$CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test
UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1"

View File

@ -8,12 +8,11 @@ SELECT sum(c0 = 0), min(c0 + 1), sum(c0 + 2) FROM t_having
GROUP BY c0 HAVING c0 = 0
SETTINGS enable_optimize_predicate_expression=0;
SET enable_positional_arguments=0;
SELECT c0 + -1, sum(intDivOrZero(intDivOrZero(NULL, NULL), '2'), intDivOrZero(10000000000., intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), 10), NULL))) FROM t_having GROUP BY c0 = 2, c0 = 10, intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), NULL), NULL), c0 HAVING c0 = 2 SETTINGS enable_optimize_predicate_expression = 0;
SELECT sum(c0 + 257) FROM t_having GROUP BY c0 = -9223372036854775808, NULL, -2147483649, c0 HAVING c0 = -9223372036854775808 SETTINGS enable_optimize_predicate_expression = 0;
SET enable_positional_arguments=0;
SELECT c0 + -2, c0 + -9223372036854775807, c0 = NULL FROM t_having GROUP BY c0 = 0.9998999834060669, 1023, c0 HAVING c0 = 0.9998999834060669 SETTINGS enable_optimize_predicate_expression = 0;
DROP TABLE t_having;

View File

@ -3,50 +3,18 @@ select x3, x2, x1 from test order by 1;
1 100 100
10 1 10
100 10 1
select x3, x2, x1 from test order by -3;
1 100 100
10 1 10
100 10 1
select x3, x2, x1 from test order by x3;
1 100 100
10 1 10
100 10 1
select x3, x2, x1 from test order by 3;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by -1;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by x1;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by 1 desc;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by -3 desc;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by x3 desc;
100 10 1
10 1 10
1 100 100
select x3, x2, x1 from test order by 3 desc;
1 100 100
10 1 10
100 10 1
select x3, x2, x1 from test order by -1 desc;
1 100 100
10 1 10
100 10 1
select x3, x2, x1 from test order by x1 desc;
1 100 100
10 1 10
100 10 1
insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1);
select x3, x2 from test group by x3, x2 order by x3;
1 100
@ -86,20 +54,6 @@ SELECT
x1
FROM test
ORDER BY x3 + 1 ASC
explain syntax select x3, x2, x1 from test order by -1;
SELECT
x3,
x2,
x1
FROM test
ORDER BY x1 ASC
explain syntax select x3 + 1, x2, x1 from test order by -1;
SELECT
x3 + 1,
x2,
x1
FROM test
ORDER BY x1 ASC
explain syntax select x3, x3 - x2, x2, x1 from test order by 2;
SELECT
x3,
@ -108,14 +62,6 @@ SELECT
x1
FROM test
ORDER BY x3 - x2 ASC
explain syntax select x3, x3 - x2, x2, x1 from test order by -2;
SELECT
x3,
x3 - x2,
x2,
x1
FROM test
ORDER BY x2 ASC
explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2;
SELECT
x3,
@ -123,28 +69,12 @@ SELECT
x1 + x2
FROM test
ORDER BY if(x3 > 10, x3, x1 + x2) ASC
explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2;
SELECT
x3,
if(x3 > 10, x3, x1 + x2),
x1 + x2
FROM test
ORDER BY if(x3 > 10, x3, x1 + x2) ASC
explain syntax select max(x1), x2 from test group by 2 order by 1, 2;
SELECT
max(x1),
x2
FROM test
GROUP BY x2
ORDER BY
max(x1) ASC,
x2 ASC
explain syntax select max(x1), x2 from test group by -1 order by -2, -1;
SELECT
max(x1),
x2
FROM test
GROUP BY x2
ORDER BY
max(x1) ASC,
x2 ASC
@ -153,34 +83,16 @@ SELECT
1 + greatest(x1, 1),
x2
FROM test
GROUP BY
1 + greatest(x1, 1),
x2
explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1;
SELECT
1 + greatest(x1, 1),
x2
FROM test
GROUP BY
1 + greatest(x1, 1),
x2
select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 }
select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 }
select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 }
select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 }
explain syntax select x1 + x3, x3 from test group by 1, 2;
SELECT
x1 + x3,
x3
FROM test
GROUP BY
x1 + x3,
x3
explain syntax select x1 + x3, x3 from test group by -2, -1;
SELECT
x1 + x3,
x3
FROM test
GROUP BY
x1 + x3,
x3
@ -190,14 +102,8 @@ select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2,
1 2 10 100
10 20 1 10
100 200 100 1
select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc;
1 2 10 100
10 20 1 10
100 200 100 1
select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a;
44 88 13 14 15 16
select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a;
44 88 13 14 15 16
explain syntax select plus(1, 1) as a group by a;
SELECT 1 + 1 AS a
GROUP BY a

View File

@ -9,21 +9,11 @@ insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1);
-- { echo }
select x3, x2, x1 from test order by 1;
select x3, x2, x1 from test order by -3;
select x3, x2, x1 from test order by x3;
select x3, x2, x1 from test order by 3;
select x3, x2, x1 from test order by -1;
select x3, x2, x1 from test order by x1;
select x3, x2, x1 from test order by 1 desc;
select x3, x2, x1 from test order by -3 desc;
select x3, x2, x1 from test order by x3 desc;
select x3, x2, x1 from test order by 3 desc;
select x3, x2, x1 from test order by -1 desc;
select x3, x2, x1 from test order by x1 desc;
insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1);
select x3, x2 from test group by x3, x2 order by x3;
select x3, x2 from test group by 1, 2 order by x3;
@ -35,32 +25,21 @@ select x1, x2, x3 from test order by 3 limit 1 by 1;
explain syntax select x3, x2, x1 from test order by 1;
explain syntax select x3 + 1, x2, x1 from test order by 1;
explain syntax select x3, x2, x1 from test order by -1;
explain syntax select x3 + 1, x2, x1 from test order by -1;
explain syntax select x3, x3 - x2, x2, x1 from test order by 2;
explain syntax select x3, x3 - x2, x2, x1 from test order by -2;
explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2;
explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2;
explain syntax select max(x1), x2 from test group by 2 order by 1, 2;
explain syntax select max(x1), x2 from test group by -1 order by -2, -1;
explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2;
explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1;
select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 }
select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 }
select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 }
select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 }
explain syntax select x1 + x3, x3 from test group by 1, 2;
explain syntax select x1 + x3, x3 from test group by -2, -1;
create table test2(x1 Int, x2 Int, x3 Int) engine=Memory;
insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1);
select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc;
select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc;
select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a;
select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a;
explain syntax select plus(1, 1) as a group by a;
select substr('aaaaaaaaaaaaaa', 8) as a group by a order by a;

View File

@ -1,6 +1,5 @@
-- https://github.com/ClickHouse/ClickHouse/issues/43202
-- Queries are generated by the fuzzer, so don't expect them to make sense
SET enable_positional_arguments=0;
SELECT NULL, '' FROM (SELECT toNullable(''), NULL AS key GROUP BY GROUPING SETS ((NULL))) AS s1 ALL LEFT JOIN (SELECT '' AS key, NULL AS value GROUP BY GROUPING SETS (('')) WITH TOTALS UNION ALL SELECT NULL AS key, toNullable(NULL) AS value GROUP BY '', NULL, '' WITH TOTALS) AS s2 USING (key);
SELECT NULL GROUP BY NULL WITH TOTALS;
SELECT 1048575, NULL, b FROM (SELECT '25.5' AS a, NULL, NULL AS b GROUP BY GROUPING SETS ((0.0001)) WITH TOTALS) AS js1 ANY RIGHT JOIN (SELECT NULL AS a, NULL AS b WHERE NULL GROUP BY NULL, -9223372036854775807 WITH CUBE WITH TOTALS UNION ALL SELECT NULL AS a, NULL AS b GROUP BY 1, '21474836.46' WITH TOTALS) AS js2 USING (a, b) ORDER BY nan DESC NULLS LAST, '9223372036854775807' DESC NULLS LAST, a ASC NULLS LAST;

View File

@ -0,0 +1,2 @@
45 1
processed 99 0

View File

@ -0,0 +1,23 @@
-- https://github.com/ClickHouse/ClickHouse/issues/46628
DROP TABLE IF EXISTS t;
CREATE TABLE t
(
`n` int
)
ENGINE = MergeTree
ORDER BY n AS
SELECT *
FROM numbers(10);
SELECT
sum(n),
1 AS x
FROM t
GROUP BY x;
SELECT
'processed' AS type,
max(number) AS max_date,
min(number) AS min_date
FROM numbers(100)
GROUP BY type;