mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge remote-tracking branch 'upstream/master' into issue-6459
This commit is contained in:
commit
7285bad1e7
@ -17,7 +17,7 @@ A clear and concise description of what works not as it is supposed to.
|
||||
* Which interface to use, if matters
|
||||
* Non-default settings, if any
|
||||
* `CREATE TABLE` statements for all tables involved
|
||||
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
|
||||
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/ClickHouse/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
|
||||
* Queries to run that lead to unexpected result
|
||||
|
||||
**Expected behavior**
|
||||
|
2
.github/ISSUE_TEMPLATE/performance-issue.md
vendored
2
.github/ISSUE_TEMPLATE/performance-issue.md
vendored
@ -17,7 +17,7 @@ What exactly works slower than expected?
|
||||
* Which interface to use, if matters
|
||||
* Non-default settings, if any
|
||||
* `CREATE TABLE` statements for all tables involved
|
||||
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
|
||||
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/ClickHouse/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
|
||||
* Queries to run that lead to slow performance
|
||||
|
||||
**Expected performance**
|
||||
|
2766
CHANGELOG.md
2766
CHANGELOG.md
File diff suppressed because it is too large
Load Diff
@ -1,4 +1,4 @@
|
||||
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/yandex/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.yandex)
|
||||
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.yandex)
|
||||
|
||||
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
|
||||
|
||||
|
@ -14,4 +14,4 @@ currently being supported with security updates:
|
||||
## Reporting a Vulnerability
|
||||
|
||||
To report a potential vulnerability in ClickHouse please use the security advisory feature of GitHub:
|
||||
https://github.com/yandex/ClickHouse/security/advisories
|
||||
https://github.com/ClickHouse/ClickHouse/security/advisories
|
||||
|
@ -14,8 +14,11 @@ if (SANITIZE)
|
||||
endif ()
|
||||
|
||||
elseif (SANITIZE STREQUAL "memory")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=memory -fsanitize-memory-track-origins")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=memory -fsanitize-memory-track-origins")
|
||||
set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-track-origins -fno-optimize-sibling-calls")
|
||||
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}")
|
||||
|
||||
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=memory")
|
||||
endif()
|
||||
@ -23,6 +26,28 @@ if (SANITIZE)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libmsan")
|
||||
endif ()
|
||||
|
||||
# Temporarily disable many external libraries that don't work under
|
||||
# MemorySanitizer yet.
|
||||
set (ENABLE_HDFS 0 CACHE BOOL "")
|
||||
set (ENABLE_CAPNP 0 CACHE BOOL "")
|
||||
set (ENABLE_RDKAFKA 0 CACHE BOOL "")
|
||||
set (ENABLE_ICU 0 CACHE BOOL "")
|
||||
set (ENABLE_POCO_MONGODB 0 CACHE BOOL "")
|
||||
set (ENABLE_POCO_NETSSL 0 CACHE BOOL "")
|
||||
set (ENABLE_POCO_ODBC 0 CACHE BOOL "")
|
||||
set (ENABLE_ODBC 0 CACHE BOOL "")
|
||||
set (ENABLE_MYSQL 0 CACHE BOOL "")
|
||||
set (ENABLE_EMBEDDED_COMPILER 0 CACHE BOOL "")
|
||||
set (USE_INTERNAL_CAPNP_LIBRARY 0 CACHE BOOL "")
|
||||
set (USE_SIMDJSON 0 CACHE BOOL "")
|
||||
set (ENABLE_READLINE 0 CACHE BOOL "")
|
||||
set (ENABLE_ORC 0 CACHE BOOL "")
|
||||
set (ENABLE_PARQUET 0 CACHE BOOL "")
|
||||
set (USE_CAPNP 0 CACHE BOOL "")
|
||||
set (USE_INTERNAL_ORC_LIBRARY 0 CACHE BOOL "")
|
||||
set (USE_ORC 0 CACHE BOOL "")
|
||||
set (ENABLE_SSL 0 CACHE BOOL "")
|
||||
|
||||
elseif (SANITIZE STREQUAL "thread")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=thread")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=thread")
|
||||
|
@ -1,12 +1,12 @@
|
||||
if (CMAKE_SYSTEM_NAME MATCHES "Linux")
|
||||
set (OS_LINUX 1)
|
||||
add_compile_definitions(OS_LINUX)
|
||||
add_definitions(-D OS_LINUX)
|
||||
elseif (CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
|
||||
set (OS_FREEBSD 1)
|
||||
add_compile_definitions(OS_FREEBSD)
|
||||
add_definitions(-D OS_FREEBSD)
|
||||
elseif (CMAKE_SYSTEM_NAME MATCHES "Darwin")
|
||||
set (OS_DARWIN 1)
|
||||
add_compile_definitions(OS_DARWIN)
|
||||
add_definitions(-D OS_DARWIN)
|
||||
endif ()
|
||||
|
||||
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
|
||||
|
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54426)
|
||||
set(VERSION_REVISION 54427)
|
||||
set(VERSION_MAJOR 19)
|
||||
set(VERSION_MINOR 15)
|
||||
set(VERSION_MINOR 16)
|
||||
set(VERSION_PATCH 1)
|
||||
set(VERSION_GITHASH 6f1a8c37abe6ee4e7ee74c0b5cb9c05a87417b61)
|
||||
set(VERSION_DESCRIBE v19.15.1.1-prestable)
|
||||
set(VERSION_STRING 19.15.1.1)
|
||||
set(VERSION_GITHASH 38f65a6a2120d2e76bcf71131068f41195149dfc)
|
||||
set(VERSION_DESCRIBE v19.16.1.1-prestable)
|
||||
set(VERSION_STRING 19.16.1.1)
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -1,10 +1,12 @@
|
||||
#include <Common/DiskSpaceMonitor.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <set>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -45,7 +47,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path)
|
||||
return absolute_path;
|
||||
}
|
||||
|
||||
/// Returns name of filesystem mounted to mount_point
|
||||
/// Returns name of filesystem mounted to mount_point
|
||||
#if !defined(__linux__)
|
||||
[[noreturn]]
|
||||
#endif
|
||||
@ -65,7 +67,7 @@ std::string getFilesystemName([[maybe_unused]] const std::string & mount_point)
|
||||
throw DB::Exception("Cannot find name of filesystem by mount point " + mount_point, ErrorCodes::SYSTEM_ERROR);
|
||||
return fs_info.mnt_fsname;
|
||||
#else
|
||||
throw DB::Exception("Supported on linux only", ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw DB::Exception("The function getFilesystemName is supported on Linux only", ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
|
||||
@ -82,7 +84,7 @@ bool Disk::tryReserve(UInt64 bytes) const
|
||||
std::lock_guard lock(mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskSpaceMonitor"), "Reserving 0 bytes on disk " << name);
|
||||
LOG_DEBUG(&Logger::get("DiskSpaceMonitor"), "Reserving 0 bytes on disk " << backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
@ -93,7 +95,8 @@ bool Disk::tryReserve(UInt64 bytes) const
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskSpaceMonitor"),
|
||||
"Reserving " << bytes << " bytes on disk " << name << " having unreserved " << unreserved_space << " bytes.");
|
||||
"Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name)
|
||||
<< ", having unreserved " << formatReadableSizeWithBinarySuffix(unreserved_space) << ".");
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return true;
|
||||
@ -283,14 +286,14 @@ Volume::Volume(
|
||||
max_data_part_size = static_cast<decltype(max_data_part_size)>(sum_size * ratio / disks.size());
|
||||
for (size_t i = 0; i < disks.size(); ++i)
|
||||
if (sizes[i] < max_data_part_size)
|
||||
LOG_WARNING(logger, "Disk " << disks[i]->getName() << " on volume " << config_prefix <<
|
||||
" have not enough space (" << sizes[i] <<
|
||||
LOG_WARNING(logger, "Disk " << backQuote(disks[i]->getName()) << " on volume " << backQuote(config_prefix) <<
|
||||
" have not enough space (" << formatReadableSizeWithBinarySuffix(sizes[i]) <<
|
||||
") for containing part the size of max_data_part_size (" <<
|
||||
max_data_part_size << ")");
|
||||
formatReadableSizeWithBinarySuffix(max_data_part_size) << ")");
|
||||
}
|
||||
constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
|
||||
if (max_data_part_size < MIN_PART_SIZE)
|
||||
LOG_WARNING(logger, "Volume '" << name << "' max_data_part_size is too low ("
|
||||
LOG_WARNING(logger, "Volume " << backQuote(name) << " max_data_part_size is too low ("
|
||||
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << " < "
|
||||
<< formatReadableSizeWithBinarySuffix(MIN_PART_SIZE) << ")");
|
||||
}
|
||||
@ -505,7 +508,7 @@ StoragePolicySelector::StoragePolicySelector(
|
||||
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
policies.emplace(name, std::make_shared<StoragePolicy>(name, config, config_prefix + "." + name, disks));
|
||||
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy " << name << " loaded");
|
||||
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy " << backQuote(name) << " loaded");
|
||||
}
|
||||
|
||||
constexpr auto default_storage_policy_name = "default";
|
||||
|
@ -457,6 +457,7 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_PROTOCOL = 480;
|
||||
extern const int PATH_ACCESS_DENIED = 481;
|
||||
extern const int DICTIONARY_ACCESS_DENIED = 482;
|
||||
extern const int TOO_MANY_REDIRECTS = 483;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
size_t wipeSensitiveData(std::string & data) const;
|
||||
|
||||
/// setInstance is not thread-safe and should be called once in single-thread mode.
|
||||
/// https://github.com/yandex/ClickHouse/pull/6810#discussion_r321183367
|
||||
/// https://github.com/ClickHouse/ClickHouse/pull/6810#discussion_r321183367
|
||||
static void setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_);
|
||||
static SensitiveDataMasker * getInstance();
|
||||
|
||||
|
@ -35,7 +35,9 @@ void checkStackSize()
|
||||
// Stack size for the main thread is 8MB on OSX excluding the guard page size.
|
||||
pthread_t thread = pthread_self();
|
||||
max_stack_size = pthread_main_np() ? (8 * 1024 * 1024) : pthread_get_stacksize_np(thread);
|
||||
stack_address = pthread_get_stackaddr_np(thread);
|
||||
|
||||
// stack_address points to the start of the stack, not the end how it's returned by pthread_get_stackaddr_np
|
||||
stack_address = reinterpret_cast<void*>(reinterpret_cast<uintptr_t>(pthread_get_stackaddr_np(thread)) - max_stack_size);
|
||||
#else
|
||||
pthread_attr_t attr;
|
||||
# if defined(__FreeBSD__)
|
||||
|
@ -219,11 +219,14 @@ size_t Block::getPositionByName(const std::string & name) const
|
||||
}
|
||||
|
||||
|
||||
void Block::checkNumberOfRows() const
|
||||
void Block::checkNumberOfRows(bool allow_null_columns) const
|
||||
{
|
||||
ssize_t rows = -1;
|
||||
for (const auto & elem : data)
|
||||
{
|
||||
if (!elem.column && allow_null_columns)
|
||||
continue;
|
||||
|
||||
if (!elem.column)
|
||||
throw Exception("Column " + elem.name + " in block is nullptr, in method checkNumberOfRows."
|
||||
, ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
@ -90,7 +90,7 @@ public:
|
||||
size_t columns() const { return data.size(); }
|
||||
|
||||
/// Checks that every column in block is not nullptr and has same number of elements.
|
||||
void checkNumberOfRows() const;
|
||||
void checkNumberOfRows(bool allow_null_columns = false) const;
|
||||
|
||||
/// Approximate number of bytes in memory - for profiling and limits.
|
||||
size_t bytes() const;
|
||||
|
@ -170,6 +170,8 @@ struct Settings : public SettingsCollection<Settings>
|
||||
\
|
||||
M(SettingBool, add_http_cors_header, false, "Write add http CORS header.") \
|
||||
\
|
||||
M(SettingUInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.") \
|
||||
\
|
||||
M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).") \
|
||||
M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.") \
|
||||
M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).") \
|
||||
|
@ -946,9 +946,10 @@ public:
|
||||
{
|
||||
WhichDataType which(arguments[0]);
|
||||
|
||||
if (!which.isStringOrFixedString()
|
||||
&& !which.isDateOrDateTime()
|
||||
&& !which.isUInt())
|
||||
if (!which.isStringOrFixedString() &&
|
||||
!which.isDateOrDateTime() &&
|
||||
!which.isUInt() &&
|
||||
!which.isFloat())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1021,6 +1022,45 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool tryExecuteFloat(const IColumn * col, ColumnPtr & col_res)
|
||||
{
|
||||
const ColumnVector<T> * col_vec = checkAndGetColumn<ColumnVector<T>>(col);
|
||||
|
||||
static constexpr size_t FLOAT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte.
|
||||
|
||||
if (col_vec)
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const typename ColumnVector<T>::Container & in_vec = col_vec->getData();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * FLOAT_HEX_LENGTH);
|
||||
|
||||
size_t pos = 0;
|
||||
char * out = reinterpret_cast<char *>(&out_vec[0]);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * in_pos = reinterpret_cast<const UInt8 *>(&in_vec[i]);
|
||||
executeOneString(in_pos, in_pos + sizeof(T), out);
|
||||
|
||||
pos += FLOAT_HEX_LENGTH;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out)
|
||||
{
|
||||
while (pos < end)
|
||||
@ -1135,7 +1175,9 @@ public:
|
||||
tryExecuteUInt<UInt32>(column, res_column) ||
|
||||
tryExecuteUInt<UInt64>(column, res_column) ||
|
||||
tryExecuteString(column, res_column) ||
|
||||
tryExecuteFixedString(column, res_column))
|
||||
tryExecuteFixedString(column, res_column) ||
|
||||
tryExecuteFloat<Float32>(column, res_column) ||
|
||||
tryExecuteFloat<Float64>(column, res_column))
|
||||
return;
|
||||
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
|
@ -337,19 +337,43 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
size_t num_rows = input_rows_count;
|
||||
ColumnPtr indexes;
|
||||
|
||||
/// Find first LowCardinality column and replace it to nested dictionary.
|
||||
for (auto arg : args)
|
||||
{
|
||||
ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||
if (auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
|
||||
{
|
||||
/// Single LowCardinality column is supported now.
|
||||
if (indexes)
|
||||
throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
indexes = low_cardinality_column->getIndexesPtr();
|
||||
num_rows = low_cardinality_column->getDictionary().size();
|
||||
auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(column.type.get());
|
||||
|
||||
if (!low_cardinality_type)
|
||||
throw Exception("Incompatible type for low cardinality column: " + column.type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (can_be_executed_on_default_arguments)
|
||||
{
|
||||
/// Normal case, when function can be executed on values's default.
|
||||
column.column = low_cardinality_column->getDictionary().getNestedColumn();
|
||||
indexes = low_cardinality_column->getIndexesPtr();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Special case when default value can't be used. Example: 1 % LowCardinality(Int).
|
||||
/// LowCardinality always contains default, so 1 % 0 will throw exception in normal case.
|
||||
auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size());
|
||||
column.column = dict_encoded.dictionary;
|
||||
indexes = dict_encoded.indexes;
|
||||
}
|
||||
|
||||
num_rows = column.column->size();
|
||||
column.type = low_cardinality_type->getDictionaryType();
|
||||
}
|
||||
}
|
||||
|
||||
/// Change size of constants.
|
||||
for (auto arg : args)
|
||||
{
|
||||
ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||
@ -358,26 +382,12 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
column.column = column_const->removeLowCardinality()->cloneResized(num_rows);
|
||||
column.type = removeLowCardinality(column.type);
|
||||
}
|
||||
else if (auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
|
||||
{
|
||||
auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(column.type.get());
|
||||
|
||||
if (!low_cardinality_type)
|
||||
throw Exception("Incompatible type for low cardinality column: " + column.type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (can_be_executed_on_default_arguments)
|
||||
column.column = low_cardinality_column->getDictionary().getNestedColumn();
|
||||
else
|
||||
{
|
||||
auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size());
|
||||
column.column = dict_encoded.dictionary;
|
||||
indexes = dict_encoded.indexes;
|
||||
}
|
||||
column.type = low_cardinality_type->getDictionaryType();
|
||||
}
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
block.checkNumberOfRows(true);
|
||||
#endif
|
||||
|
||||
return indexes;
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@ namespace DB
|
||||
{
|
||||
class FunctionFactory;
|
||||
|
||||
void registerFunctionRepeat(FunctionFactory &);
|
||||
void registerFunctionEmpty(FunctionFactory &);
|
||||
void registerFunctionNotEmpty(FunctionFactory &);
|
||||
void registerFunctionLength(FunctionFactory &);
|
||||
@ -34,6 +35,7 @@ void registerFunctionTryBase64Decode(FunctionFactory &);
|
||||
|
||||
void registerFunctionsString(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionRepeat(factory);
|
||||
registerFunctionEmpty(factory);
|
||||
registerFunctionNotEmpty(factory);
|
||||
registerFunctionLength(factory);
|
||||
|
212
dbms/src/Functions/repeat.cpp
Normal file
212
dbms/src/Functions/repeat.cpp
Normal file
@ -0,0 +1,212 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
struct RepeatImpl
|
||||
{
|
||||
/// Safety threshold against DoS.
|
||||
static inline void checkRepeatTime(UInt64 repeat_time)
|
||||
{
|
||||
static constexpr UInt64 max_repeat_times = 1000000;
|
||||
if (repeat_time > max_repeat_times)
|
||||
throw Exception("Too many times to repeat (" + std::to_string(repeat_time) + "), maximum is: " + std::to_string(max_repeat_times),
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
}
|
||||
|
||||
static void vectorStrConstRepeat(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars & res_data,
|
||||
ColumnString::Offsets & res_offsets,
|
||||
UInt64 repeat_time)
|
||||
{
|
||||
checkRepeatTime(repeat_time);
|
||||
|
||||
UInt64 data_size = 0;
|
||||
res_offsets.assign(offsets);
|
||||
for (UInt64 i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
data_size += (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1; /// Note that accessing -1th element is valid for PaddedPODArray.
|
||||
res_offsets[i] = data_size;
|
||||
}
|
||||
res_data.resize(data_size);
|
||||
for (UInt64 i = 0; i < res_offsets.size(); ++i)
|
||||
{
|
||||
process(data.data() + offsets[i - 1], res_data.data() + res_offsets[i - 1], offsets[i] - offsets[i - 1], repeat_time);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void vectorStrVectorRepeat(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars & res_data,
|
||||
ColumnString::Offsets & res_offsets,
|
||||
const PaddedPODArray<T> & col_num)
|
||||
{
|
||||
UInt64 data_size = 0;
|
||||
res_offsets.assign(offsets);
|
||||
for (UInt64 i = 0; i < col_num.size(); ++i)
|
||||
{
|
||||
data_size += (offsets[i] - offsets[i - 1] - 1) * col_num[i] + 1;
|
||||
res_offsets[i] = data_size;
|
||||
}
|
||||
res_data.resize(data_size);
|
||||
|
||||
for (UInt64 i = 0; i < col_num.size(); ++i)
|
||||
{
|
||||
T repeat_time = col_num[i];
|
||||
checkRepeatTime(repeat_time);
|
||||
process(data.data() + offsets[i - 1], res_data.data() + res_offsets[i - 1], offsets[i] - offsets[i - 1], repeat_time);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void constStrVectorRepeat(
|
||||
const StringRef & copy_str,
|
||||
ColumnString::Chars & res_data,
|
||||
ColumnString::Offsets & res_offsets,
|
||||
const PaddedPODArray<T> & col_num)
|
||||
{
|
||||
UInt64 data_size = 0;
|
||||
res_offsets.resize(col_num.size());
|
||||
UInt64 str_size = copy_str.size;
|
||||
UInt64 col_size = col_num.size();
|
||||
for (UInt64 i = 0; i < col_size; ++i)
|
||||
{
|
||||
data_size += str_size * col_num[i] + 1;
|
||||
res_offsets[i] = data_size;
|
||||
}
|
||||
res_data.resize(data_size);
|
||||
for (UInt64 i = 0; i < col_size; ++i)
|
||||
{
|
||||
T repeat_time = col_num[i];
|
||||
checkRepeatTime(repeat_time);
|
||||
process(
|
||||
reinterpret_cast<UInt8 *>(const_cast<char *>(copy_str.data)),
|
||||
res_data.data() + res_offsets[i - 1],
|
||||
str_size + 1,
|
||||
repeat_time);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
static void process(const UInt8 * src, UInt8 * dst, UInt64 size, UInt64 repeat_time)
|
||||
{
|
||||
for (UInt64 i = 0; i < repeat_time; ++i)
|
||||
{
|
||||
memcpy(dst, src, size - 1);
|
||||
dst += size - 1;
|
||||
}
|
||||
*dst = 0;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionRepeat : public IFunction
|
||||
{
|
||||
template <typename F>
|
||||
static bool castType(const IDataType * type, F && f)
|
||||
{
|
||||
return castTypeToEither<DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64>(type, std::forward<F>(f));
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "repeat";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRepeat>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isUnsignedInteger(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
const auto & strcolumn = block.getByPosition(arguments[0]).column;
|
||||
const auto & numcolumn = block.getByPosition(arguments[1]).column;
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(strcolumn.get()))
|
||||
{
|
||||
if (const ColumnConst * scale_column_num = checkAndGetColumn<ColumnConst>(numcolumn.get()))
|
||||
{
|
||||
UInt64 repeat_time = scale_column_num->getValue<UInt64>();
|
||||
auto col_res = ColumnString::create();
|
||||
RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time);
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return;
|
||||
}
|
||||
else if (castType(block.getByPosition(arguments[1]).type.get(), [&](const auto & type)
|
||||
{
|
||||
using DataType = std::decay_t<decltype(type)>;
|
||||
using T = typename DataType::FieldType;
|
||||
const ColumnVector<T> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get());
|
||||
auto col_res = ColumnString::create();
|
||||
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->getData());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}))
|
||||
{
|
||||
return;
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(strcolumn.get()))
|
||||
{
|
||||
/// Note that const-const case is handled by useDefaultImplementationForConstants.
|
||||
|
||||
StringRef copy_str = col_const->getDataColumn().getDataAt(0);
|
||||
|
||||
if (castType(block.getByPosition(arguments[1]).type.get(), [&](const auto & type)
|
||||
{
|
||||
using DataType = std::decay_t<decltype(type)>;
|
||||
using T = typename DataType::FieldType;
|
||||
const ColumnVector<T> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get());
|
||||
auto col_res = ColumnString::create();
|
||||
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->getData());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}))
|
||||
{
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionRepeat(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionRepeat>();
|
||||
}
|
||||
|
||||
}
|
@ -17,8 +17,8 @@ HDFSBuilderPtr createHDFSBuilder(const std::string & uri_str)
|
||||
const Poco::URI uri(uri_str);
|
||||
auto & host = uri.getHost();
|
||||
auto port = uri.getPort();
|
||||
auto & path = uri.getPath();
|
||||
if (host.empty() || path.empty())
|
||||
const std::string path = "//";
|
||||
if (host.empty())
|
||||
throw Exception("Illegal HDFS URI: " + uri.toString(), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
HDFSBuilderPtr builder(hdfsNewBuilder());
|
||||
|
@ -40,6 +40,7 @@ namespace ErrorCodes
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
|
||||
extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME;
|
||||
extern const int UNSUPPORTED_URI_SCHEME;
|
||||
extern const int TOO_MANY_REDIRECTS;
|
||||
}
|
||||
|
||||
|
||||
@ -216,20 +217,21 @@ PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Connecti
|
||||
return HTTPSessionPool::instance().getSession(uri, timeouts, per_endpoint_pool_size);
|
||||
}
|
||||
|
||||
bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; }
|
||||
|
||||
std::istream * receiveResponse(
|
||||
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response)
|
||||
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, const bool allow_redirects)
|
||||
{
|
||||
auto & istr = session.receiveResponse(response);
|
||||
assertResponseIsOk(request, response, istr);
|
||||
assertResponseIsOk(request, response, istr, allow_redirects);
|
||||
return &istr;
|
||||
}
|
||||
|
||||
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr)
|
||||
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects)
|
||||
{
|
||||
auto status = response.getStatus();
|
||||
|
||||
if (status != Poco::Net::HTTPResponse::HTTP_OK)
|
||||
if (!(status == Poco::Net::HTTPResponse::HTTP_OK || (isRedirect(status) && allow_redirects)))
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " "
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <Common/PoolBase.h>
|
||||
|
||||
#include <Poco/URIStreamFactory.h>
|
||||
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
|
||||
@ -50,13 +50,14 @@ HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts &
|
||||
/// As previous method creates session, but tooks it from pool
|
||||
PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size);
|
||||
|
||||
bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status);
|
||||
|
||||
/** Used to receive response (response headers and possibly body)
|
||||
* after sending data (request headers and possibly body).
|
||||
* Throws exception in case of non HTTP_OK (200) response code.
|
||||
* Returned istream lives in 'session' object.
|
||||
*/
|
||||
std::istream * receiveResponse(
|
||||
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response);
|
||||
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr);
|
||||
|
||||
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects);
|
||||
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects = false);
|
||||
}
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/config.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/URIStreamFactory.h>
|
||||
|
||||
|
||||
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
|
||||
@ -26,41 +27,84 @@ namespace DB
|
||||
/** Perform HTTP POST request and provide response to read.
|
||||
*/
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TOO_MANY_REDIRECTS;
|
||||
}
|
||||
|
||||
template <typename SessionPtr>
|
||||
class UpdatableSessionBase
|
||||
{
|
||||
protected:
|
||||
SessionPtr session;
|
||||
UInt64 redirects { 0 };
|
||||
Poco::URI initial_uri;
|
||||
const ConnectionTimeouts & timeouts;
|
||||
DB::SettingUInt64 max_redirects;
|
||||
|
||||
public:
|
||||
virtual void buildNewSession(const Poco::URI & uri) = 0;
|
||||
|
||||
explicit UpdatableSessionBase(const Poco::URI uri,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
SettingUInt64 max_redirects_)
|
||||
: initial_uri { uri }
|
||||
, timeouts { timeouts_ }
|
||||
, max_redirects { max_redirects_ }
|
||||
{
|
||||
}
|
||||
|
||||
SessionPtr getSession()
|
||||
{
|
||||
return session;
|
||||
}
|
||||
|
||||
void updateSession(const Poco::URI & uri)
|
||||
{
|
||||
++redirects;
|
||||
if (redirects <= max_redirects)
|
||||
{
|
||||
buildNewSession(uri);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Too many redirects while trying to access " << initial_uri.toString();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::TOO_MANY_REDIRECTS);
|
||||
}
|
||||
}
|
||||
|
||||
virtual ~UpdatableSessionBase()
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
namespace detail
|
||||
{
|
||||
template <typename SessionPtr>
|
||||
template <typename UpdatableSessionPtr>
|
||||
class ReadWriteBufferFromHTTPBase : public ReadBuffer
|
||||
{
|
||||
protected:
|
||||
Poco::URI uri;
|
||||
std::string method;
|
||||
|
||||
SessionPtr session;
|
||||
UpdatableSessionPtr session;
|
||||
std::istream * istr; /// owned by session
|
||||
std::unique_ptr<ReadBuffer> impl;
|
||||
std::function<void(std::ostream &)> out_stream_callback;
|
||||
const Poco::Net::HTTPBasicCredentials & credentials;
|
||||
std::vector<Poco::Net::HTTPCookie> cookies;
|
||||
|
||||
public:
|
||||
using OutStreamCallback = std::function<void(std::ostream &)>;
|
||||
|
||||
explicit ReadWriteBufferFromHTTPBase(
|
||||
SessionPtr session_,
|
||||
Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0)
|
||||
, uri{uri_}
|
||||
, method{!method_.empty() ? method_ : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
|
||||
, session{session_}
|
||||
std::istream * call(const Poco::URI uri_, Poco::Net::HTTPResponse & response)
|
||||
{
|
||||
// With empty path poco will send "POST HTTP/1.1" its bug.
|
||||
if (uri.getPath().empty())
|
||||
uri.setPath("/");
|
||||
|
||||
Poco::Net::HTTPRequest request(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
|
||||
request.setHost(uri.getHost()); // use original, not resolved host name in header
|
||||
Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
|
||||
request.setHost(uri_.getHost()); // use original, not resolved host name in header
|
||||
|
||||
if (out_stream_callback)
|
||||
request.setChunkedTransferEncoding(true);
|
||||
@ -68,27 +112,71 @@ namespace detail
|
||||
if (!credentials.getUsername().empty())
|
||||
credentials.authenticate(request);
|
||||
|
||||
Poco::Net::HTTPResponse response;
|
||||
|
||||
LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to " << uri.toString());
|
||||
|
||||
auto sess = session->getSession();
|
||||
|
||||
try
|
||||
{
|
||||
auto & stream_out = session->sendRequest(request);
|
||||
auto & stream_out = sess->sendRequest(request);
|
||||
|
||||
if (out_stream_callback)
|
||||
out_stream_callback(stream_out);
|
||||
|
||||
istr = receiveResponse(*session, request, response);
|
||||
istr = receiveResponse(*sess, request, response, true);
|
||||
response.getCookies(cookies);
|
||||
|
||||
return istr;
|
||||
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
/// We use session data storage as storage for exception text
|
||||
/// Depend on it we can deduce to reconnect session or reresolve session host
|
||||
sess->attachSessionData(e.message());
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
using OutStreamCallback = std::function<void(std::ostream &)>;
|
||||
|
||||
explicit ReadWriteBufferFromHTTPBase(UpdatableSessionPtr session_,
|
||||
Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback_ = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0)
|
||||
, uri {uri_}
|
||||
, method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
|
||||
, session {session_}
|
||||
, out_stream_callback {out_stream_callback_}
|
||||
, credentials {credentials_}
|
||||
{
|
||||
Poco::Net::HTTPResponse response;
|
||||
|
||||
istr = call(uri, response);
|
||||
|
||||
while (isRedirect(response.getStatus()))
|
||||
{
|
||||
Poco::URI uri_redirect(response.get("Location"));
|
||||
|
||||
session->updateSession(uri_redirect);
|
||||
|
||||
istr = call(uri_redirect,response);
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
/// We use session data storage as storage for exception text
|
||||
/// Depend on it we can deduce to reconnect session or reresolve session host
|
||||
session->attachSessionData(e.message());
|
||||
auto sess = session->getSession();
|
||||
sess->attachSessionData(e.message());
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -112,42 +200,88 @@ namespace detail
|
||||
};
|
||||
}
|
||||
|
||||
class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<HTTPSessionPtr>
|
||||
class UpdatableSession : public UpdatableSessionBase<HTTPSessionPtr>
|
||||
{
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<HTTPSessionPtr>;
|
||||
using Parent = UpdatableSessionBase<HTTPSessionPtr>;
|
||||
|
||||
public:
|
||||
explicit UpdatableSession(const Poco::URI uri,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const SettingUInt64 max_redirects_)
|
||||
: Parent(uri, timeouts_, max_redirects_)
|
||||
{
|
||||
session = makeHTTPSession(initial_uri, timeouts);
|
||||
}
|
||||
|
||||
void buildNewSession(const Poco::URI & uri) override
|
||||
{
|
||||
session = makeHTTPSession(uri, timeouts);
|
||||
}
|
||||
};
|
||||
|
||||
class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>
|
||||
{
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>;
|
||||
|
||||
public:
|
||||
explicit ReadWriteBufferFromHTTP(Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
OutStreamCallback out_stream_callback_ = {},
|
||||
const ConnectionTimeouts & timeouts = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials = {},
|
||||
const DB::SettingUInt64 max_redirects = 0,
|
||||
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: Parent(makeHTTPSession(uri_, timeouts), uri_, method_, out_stream_callback, credentials, buffer_size_)
|
||||
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects), uri_, method_, out_stream_callback_, credentials_, buffer_size_)
|
||||
{
|
||||
}
|
||||
};
|
||||
class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<PooledHTTPSessionPtr>
|
||||
|
||||
class UpdatablePooledSession : public UpdatableSessionBase<PooledHTTPSessionPtr>
|
||||
{
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<PooledHTTPSessionPtr>;
|
||||
using Parent = UpdatableSessionBase<PooledHTTPSessionPtr>;
|
||||
|
||||
private:
|
||||
size_t per_endpoint_pool_size;
|
||||
|
||||
public:
|
||||
explicit UpdatablePooledSession(const Poco::URI uri,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const SettingUInt64 max_redirects_,
|
||||
size_t per_endpoint_pool_size_)
|
||||
: Parent(uri, timeouts_, max_redirects_)
|
||||
, per_endpoint_pool_size { per_endpoint_pool_size_ }
|
||||
{
|
||||
session = makePooledHTTPSession(initial_uri, timeouts, per_endpoint_pool_size);
|
||||
}
|
||||
|
||||
void buildNewSession(const Poco::URI & uri) override
|
||||
{
|
||||
session = makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size);
|
||||
}
|
||||
};
|
||||
|
||||
class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatablePooledSession>>
|
||||
{
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatablePooledSession>>;
|
||||
|
||||
public:
|
||||
explicit PooledReadWriteBufferFromHTTP(Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
const ConnectionTimeouts & timeouts = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials = {},
|
||||
OutStreamCallback out_stream_callback_ = {},
|
||||
const ConnectionTimeouts & timeouts_ = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const DB::SettingUInt64 max_redirects = 0,
|
||||
size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT)
|
||||
: Parent(makePooledHTTPSession(uri_, timeouts, max_connections_per_endpoint),
|
||||
: Parent(std::make_shared<UpdatablePooledSession>(uri_, timeouts_, max_redirects, max_connections_per_endpoint),
|
||||
uri_,
|
||||
method_,
|
||||
out_stream_callback,
|
||||
credentials,
|
||||
out_stream_callback_,
|
||||
credentials_,
|
||||
buffer_size_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@ namespace ErrorCodes
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_FSYNC;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
@ -32,10 +33,12 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const std::string path = hdfs_uri.substr(begin_of_path);
|
||||
if (path.find("*?{") != std::string::npos)
|
||||
if (path.find_first_of("*?{") != std::string::npos)
|
||||
throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0);
|
||||
if (!hdfsExists(fs.get(), path.c_str()))
|
||||
throw Exception("File: " + path + " is already exists", ErrorCodes::BAD_ARGUMENTS);
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here
|
||||
|
||||
if (fout == nullptr)
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP(
|
||||
|
||||
void WriteBufferFromHTTP::finalize()
|
||||
{
|
||||
receiveResponse(*session, request, response);
|
||||
receiveResponse(*session, request, response, false);
|
||||
/// TODO: Response body is ignored.
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,10 @@ namespace DB
|
||||
{
|
||||
|
||||
const int DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT = 2;
|
||||
|
||||
// S3 protocol does not allow to have multipart upload with more than 10000 parts.
|
||||
// In case server does not return an error on exceeding that number, we print a warning
|
||||
// because custom S3 implementation may allow relaxed requirements on that.
|
||||
const int S3_WARN_MAX_PARTS = 10000;
|
||||
|
||||
|
||||
@ -166,7 +170,7 @@ void WriteBufferFromS3::writePart(const String & data)
|
||||
if (part_tags.size() == S3_WARN_MAX_PARTS)
|
||||
{
|
||||
// Don't throw exception here by ourselves but leave the decision to take by S3 server.
|
||||
LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached (too much parts). Server may not accept this whole upload.");
|
||||
LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached (too many parts). Server may not accept this whole upload.");
|
||||
}
|
||||
|
||||
for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i)
|
||||
|
@ -1,25 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <Core/Types.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromIStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Poco/Net/HTTPBasicCredentials.h>
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <Poco/Version.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/config.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -36,6 +27,9 @@ private:
|
||||
String buffer_string;
|
||||
std::unique_ptr<WriteBufferFromString> temporary_buffer;
|
||||
size_t last_part_size;
|
||||
|
||||
/// Upload in S3 is made in parts.
|
||||
/// We initiate upload, then upload each part and get ETag as a response, and then finish upload with listing all our parts.
|
||||
String upload_id;
|
||||
std::vector<String> part_tags;
|
||||
|
||||
|
@ -67,4 +67,25 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac
|
||||
writeException(Exception(Exception::CreateFromPoco, *e.nested()), buf, with_stack_trace);
|
||||
}
|
||||
|
||||
|
||||
String backQuoteIfNeed(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
{
|
||||
WriteBufferFromString wb(res);
|
||||
writeProbablyBackQuotedString(x, wb);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
String backQuote(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
{
|
||||
WriteBufferFromString wb(res);
|
||||
writeBackQuotedString(x, wb);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -906,4 +906,10 @@ inline String toString(const T & x)
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
|
||||
/// Quote the identifier with backquotes, if required.
|
||||
String backQuoteIfNeed(const String & x);
|
||||
/// Quote the identifier with backquotes.
|
||||
String backQuote(const String & x);
|
||||
|
||||
}
|
||||
|
@ -18,8 +18,6 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
|
||||
{
|
||||
Settings new_settings = settings;
|
||||
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);
|
||||
new_settings.connection_pool_max_wait_ms = Cluster::saturate(new_settings.connection_pool_max_wait_ms, settings.max_execution_time);
|
||||
new_settings.replace_running_query_max_wait_ms = Cluster::saturate(new_settings.replace_running_query_max_wait_ms, settings.max_execution_time);
|
||||
|
||||
/// Does not matter on remote servers, because queries are sent under different user.
|
||||
new_settings.max_concurrent_queries_for_user = 0;
|
||||
@ -39,8 +37,8 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
|
||||
}
|
||||
|
||||
BlockInputStreams executeQuery(
|
||||
IStreamFactory & stream_factory, const ClusterPtr & cluster,
|
||||
const ASTPtr & query_ast, const Context & context, const Settings & settings)
|
||||
IStreamFactory & stream_factory, const ClusterPtr & cluster,
|
||||
const ASTPtr & query_ast, const Context & context, const Settings & settings)
|
||||
{
|
||||
BlockInputStreams res;
|
||||
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
/// Could be called from different threads in parallel.
|
||||
virtual void joinBlock(Block & block) = 0;
|
||||
|
||||
virtual bool hasTotals() const { return false; }
|
||||
virtual bool hasTotals() const = 0;
|
||||
virtual void setTotals(const Block & block) = 0;
|
||||
virtual void joinTotals(Block & block) const = 0;
|
||||
|
||||
|
@ -24,6 +24,7 @@ public:
|
||||
void joinBlock(Block &) override;
|
||||
void joinTotals(Block &) const override;
|
||||
void setTotals(const Block &) override;
|
||||
bool hasTotals() const override { return totals; }
|
||||
size_t getTotalRowCount() const override { return right_blocks_row_count; }
|
||||
|
||||
private:
|
||||
|
@ -20,7 +20,7 @@ class Context;
|
||||
* - Query after optimization :
|
||||
* SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a WHERE id_1 = 1 UNION ALL SELECT id_2, name_2 FROM table_b WHERE id_2 = 1)
|
||||
* WHERE id_1 = 1
|
||||
* For more details : https://github.com/yandex/ClickHouse/pull/2015#issuecomment-374283452
|
||||
* For more details : https://github.com/ClickHouse/ClickHouse/pull/2015#issuecomment-374283452
|
||||
*/
|
||||
class PredicateExpressionsOptimizer
|
||||
{
|
||||
|
@ -242,7 +242,7 @@ private:
|
||||
// Lookups can be stored in a HashTable because it is memmovable
|
||||
// A std::variant contains a currently active type id (memmovable), together with a union of the types
|
||||
// The types are all std::unique_ptr, which contains a single pointer, which is memmovable.
|
||||
// Source: https://github.com/yandex/ClickHouse/issues/4906
|
||||
// Source: https://github.com/ClickHouse/ClickHouse/issues/4906
|
||||
Lookups lookups;
|
||||
};
|
||||
|
||||
|
@ -470,7 +470,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<K
|
||||
* 1: the intersection of the set and the range is non-empty
|
||||
* 2: the range contains elements not in the set
|
||||
*/
|
||||
BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types)
|
||||
BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types)
|
||||
{
|
||||
size_t tuple_size = indexes_mapping.size();
|
||||
|
||||
|
@ -170,7 +170,7 @@ using Sets = std::vector<SetPtr>;
|
||||
class IFunction;
|
||||
using FunctionPtr = std::shared_ptr<IFunction>;
|
||||
|
||||
/// Class for checkInRange function.
|
||||
/// Class for mayBeTrueInRange function.
|
||||
class MergeTreeSetIndex
|
||||
{
|
||||
public:
|
||||
@ -188,7 +188,7 @@ public:
|
||||
|
||||
size_t size() const { return ordered_set.at(0)->size(); }
|
||||
|
||||
BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types);
|
||||
BoolMask mayBeTrueInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types);
|
||||
|
||||
private:
|
||||
Columns ordered_set;
|
||||
|
@ -26,26 +26,6 @@ const char * IAST::hilite_substitution = "\033[1;36m";
|
||||
const char * IAST::hilite_none = "\033[0m";
|
||||
|
||||
|
||||
String backQuoteIfNeed(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
{
|
||||
WriteBufferFromString wb(res);
|
||||
writeProbablyBackQuotedString(x, wb);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
String backQuote(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
{
|
||||
WriteBufferFromString wb(res);
|
||||
writeBackQuotedString(x, wb);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
size_t IAST::size() const
|
||||
{
|
||||
size_t res = 1;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/IdentifierQuotingStyle.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/TypePromotion.h>
|
||||
#include <IO/WriteHelpers.h> /// backQuote, backQuoteIfNeed
|
||||
|
||||
#include <algorithm>
|
||||
#include <ostream>
|
||||
@ -223,9 +224,4 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/// Quote the identifier with backquotes, if required.
|
||||
String backQuoteIfNeed(const String & x);
|
||||
/// Quote the identifier with backquotes.
|
||||
String backQuote(const String & x);
|
||||
|
||||
}
|
||||
|
@ -219,6 +219,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
timeouts,
|
||||
creds,
|
||||
DBMS_DEFAULT_BUFFER_SIZE,
|
||||
0, /* no redirects */
|
||||
data_settings->replicated_max_parallel_fetches_for_host
|
||||
};
|
||||
|
||||
|
@ -886,7 +886,7 @@ String KeyCondition::toString() const
|
||||
*/
|
||||
|
||||
template <typename F>
|
||||
static BoolMask forAnyParallelogram(
|
||||
static bool forAnyParallelogram(
|
||||
size_t key_size,
|
||||
const Field * key_left,
|
||||
const Field * key_right,
|
||||
@ -942,15 +942,16 @@ static BoolMask forAnyParallelogram(
|
||||
for (size_t i = prefix_size + 1; i < key_size; ++i)
|
||||
parallelogram[i] = Range();
|
||||
|
||||
BoolMask result(false, false);
|
||||
result = result | callback(parallelogram);
|
||||
if (callback(parallelogram))
|
||||
return true;
|
||||
|
||||
/// [x1] x [y1 .. +inf)
|
||||
|
||||
if (left_bounded)
|
||||
{
|
||||
parallelogram[prefix_size] = Range(key_left[prefix_size]);
|
||||
result = result | forAnyParallelogram(key_size, key_left, key_right, true, false, parallelogram, prefix_size + 1, callback);
|
||||
if (forAnyParallelogram(key_size, key_left, key_right, true, false, parallelogram, prefix_size + 1, callback))
|
||||
return true;
|
||||
}
|
||||
|
||||
/// [x2] x (-inf .. y2]
|
||||
@ -958,14 +959,15 @@ static BoolMask forAnyParallelogram(
|
||||
if (right_bounded)
|
||||
{
|
||||
parallelogram[prefix_size] = Range(key_right[prefix_size]);
|
||||
result = result | forAnyParallelogram(key_size, key_left, key_right, false, true, parallelogram, prefix_size + 1, callback);
|
||||
if (forAnyParallelogram(key_size, key_left, key_right, false, true, parallelogram, prefix_size + 1, callback))
|
||||
return true;
|
||||
}
|
||||
|
||||
return result;
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
BoolMask KeyCondition::checkInRange(
|
||||
bool KeyCondition::mayBeTrueInRange(
|
||||
size_t used_key_size,
|
||||
const Field * left_key,
|
||||
const Field * right_key,
|
||||
@ -991,7 +993,7 @@ BoolMask KeyCondition::checkInRange(
|
||||
return forAnyParallelogram(used_key_size, left_key, right_key, true, right_bounded, key_ranges, 0,
|
||||
[&] (const std::vector<Range> & key_ranges_parallelogram)
|
||||
{
|
||||
auto res = checkInParallelogram(key_ranges_parallelogram, data_types);
|
||||
auto res = mayBeTrueInParallelogram(key_ranges_parallelogram, data_types);
|
||||
|
||||
/* std::cerr << "Parallelogram: ";
|
||||
for (size_t i = 0, size = key_ranges.size(); i != size; ++i)
|
||||
@ -1002,11 +1004,11 @@ BoolMask KeyCondition::checkInRange(
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
|
||||
Range key_range,
|
||||
MonotonicFunctionsChain & functions,
|
||||
DataTypePtr current_type)
|
||||
DataTypePtr current_type
|
||||
)
|
||||
{
|
||||
for (auto & func : functions)
|
||||
{
|
||||
@ -1039,7 +1041,7 @@ std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
|
||||
return key_range;
|
||||
}
|
||||
|
||||
BoolMask KeyCondition::checkInParallelogram(const std::vector<Range> & parallelogram, const DataTypes & data_types) const
|
||||
bool KeyCondition::mayBeTrueInParallelogram(const std::vector<Range> & parallelogram, const DataTypes & data_types) const
|
||||
{
|
||||
std::vector<BoolMask> rpn_stack;
|
||||
for (size_t i = 0; i < rpn.size(); ++i)
|
||||
@ -1087,7 +1089,7 @@ BoolMask KeyCondition::checkInParallelogram(const std::vector<Range> & parallelo
|
||||
if (!element.set_index)
|
||||
throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
rpn_stack.emplace_back(element.set_index->checkInRange(parallelogram, data_types));
|
||||
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types));
|
||||
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
@ -1122,23 +1124,22 @@ BoolMask KeyCondition::checkInParallelogram(const std::vector<Range> & parallelo
|
||||
}
|
||||
|
||||
if (rpn_stack.size() != 1)
|
||||
throw Exception("Unexpected stack size in KeyCondition::checkInRange", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Unexpected stack size in KeyCondition::mayBeTrueInRange", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return rpn_stack[0];
|
||||
return rpn_stack[0].can_be_true;
|
||||
}
|
||||
|
||||
|
||||
BoolMask KeyCondition::checkInRange(
|
||||
bool KeyCondition::mayBeTrueInRange(
|
||||
size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const
|
||||
{
|
||||
return checkInRange(used_key_size, left_key, right_key, data_types, true);
|
||||
return mayBeTrueInRange(used_key_size, left_key, right_key, data_types, true);
|
||||
}
|
||||
|
||||
|
||||
BoolMask KeyCondition::getMaskAfter(
|
||||
bool KeyCondition::mayBeTrueAfter(
|
||||
size_t used_key_size, const Field * left_key, const DataTypes & data_types) const
|
||||
{
|
||||
return checkInRange(used_key_size, left_key, nullptr, data_types, false);
|
||||
return mayBeTrueInRange(used_key_size, left_key, nullptr, data_types, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -235,17 +235,17 @@ public:
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr);
|
||||
|
||||
/// Whether the condition and its negation are (independently) feasible in the key range.
|
||||
/// Whether the condition is feasible in the key range.
|
||||
/// left_key and right_key must contain all fields in the sort_descr in the appropriate order.
|
||||
/// data_types - the types of the key columns.
|
||||
BoolMask checkInRange(size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const;
|
||||
bool mayBeTrueInRange(size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const;
|
||||
|
||||
/// Whether the condition and its negation are feasible in the direct product of single column ranges specified by `parallelogram`.
|
||||
BoolMask checkInParallelogram(const std::vector<Range> & parallelogram, const DataTypes & data_types) const;
|
||||
/// Whether the condition is feasible in the direct product of single column ranges specified by `parallelogram`.
|
||||
bool mayBeTrueInParallelogram(const std::vector<Range> & parallelogram, const DataTypes & data_types) const;
|
||||
|
||||
/// Are the condition and its negation valid in a semi-infinite (not limited to the right) key range.
|
||||
/// Is the condition valid in a semi-infinite (not limited to the right) key range.
|
||||
/// left_key must contain all the fields in the sort_descr in the appropriate order.
|
||||
BoolMask getMaskAfter(size_t used_key_size, const Field * left_key, const DataTypes & data_types) const;
|
||||
bool mayBeTrueAfter(size_t used_key_size, const Field * left_key, const DataTypes & data_types) const;
|
||||
|
||||
/// Checks that the index can not be used.
|
||||
bool alwaysUnknownOrTrue() const;
|
||||
@ -330,7 +330,7 @@ public:
|
||||
static const AtomMap atom_map;
|
||||
|
||||
private:
|
||||
BoolMask checkInRange(
|
||||
bool mayBeTrueInRange(
|
||||
size_t used_key_size,
|
||||
const Field * left_key,
|
||||
const Field * right_key,
|
||||
|
@ -269,8 +269,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
if (part->isEmpty())
|
||||
continue;
|
||||
|
||||
if (minmax_idx_condition && !minmax_idx_condition->checkInParallelogram(
|
||||
part->minmax_idx.parallelogram, data.minmax_idx_column_types).can_be_true)
|
||||
if (minmax_idx_condition && !minmax_idx_condition->mayBeTrueInParallelogram(
|
||||
part->minmax_idx.parallelogram, data.minmax_idx_column_types))
|
||||
continue;
|
||||
|
||||
if (max_block_numbers_to_read)
|
||||
@ -1200,8 +1200,8 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
||||
for (size_t i = 0; i < used_key_size; ++i)
|
||||
index[i]->get(range.begin, index_left[i]);
|
||||
|
||||
may_be_true = key_condition.getMaskAfter(
|
||||
used_key_size, index_left.data(), data.primary_key_data_types).can_be_true;
|
||||
may_be_true = key_condition.mayBeTrueAfter(
|
||||
used_key_size, index_left.data(), data.primary_key_data_types);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1214,8 +1214,8 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
||||
index[i]->get(range.end, index_right[i]);
|
||||
}
|
||||
|
||||
may_be_true = key_condition.checkInRange(
|
||||
used_key_size, index_left.data(), index_right.data(), data.primary_key_data_types).can_be_true;
|
||||
may_be_true = key_condition.mayBeTrueInRange(
|
||||
used_key_size, index_left.data(), index_right.data(), data.primary_key_data_types);
|
||||
}
|
||||
|
||||
if (!may_be_true)
|
||||
|
@ -378,11 +378,11 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
|
||||
rpn_stack.emplace_back(true, false);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected function type in BloomFilterCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (rpn_stack.size() != 1)
|
||||
throw Exception("Unexpected stack size in BloomFilterCondition::mayBeTrueOnGranule", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Unexpected stack size in KeyCondition::mayBeTrueInRange", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return rpn_stack[0].can_be_true;
|
||||
}
|
||||
|
@ -143,7 +143,7 @@ bool MergeTreeIndexConditionMinMax::mayBeTrueOnGranule(MergeTreeIndexGranulePtr
|
||||
for (const auto & range : granule->parallelogram)
|
||||
if (range.left.isNull() || range.right.isNull())
|
||||
return true;
|
||||
return condition.checkInParallelogram(granule->parallelogram, index.data_types).can_be_true;
|
||||
return condition.mayBeTrueInParallelogram(granule->parallelogram, index.data_types);
|
||||
}
|
||||
|
||||
|
||||
|
@ -45,7 +45,7 @@ void ReplicatedMergeTreePartCheckThread::start()
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::stop()
|
||||
{
|
||||
//based on discussion on https://github.com/yandex/ClickHouse/pull/1489#issuecomment-344756259
|
||||
//based on discussion on https://github.com/ClickHouse/ClickHouse/pull/1489#issuecomment-344756259
|
||||
//using the schedule pool there is no problem in case stop is called two time in row and the start multiple times
|
||||
|
||||
std::lock_guard lock(start_stop_mutex);
|
||||
|
@ -48,10 +48,11 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* Recursive directory listing with matched paths as a result.
|
||||
* Have the same method in StorageHDFS.
|
||||
*/
|
||||
std::vector<std::string> LSWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match)
|
||||
std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match)
|
||||
{
|
||||
const size_t first_glob = for_match.find_first_of("*?{");
|
||||
|
||||
@ -86,7 +87,8 @@ std::vector<std::string> LSWithRegexpMatching(const std::string & path_for_ls, c
|
||||
{
|
||||
if (re2::RE2::FullMatch(file_name, matcher))
|
||||
{
|
||||
Strings result_part = LSWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash));
|
||||
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
|
||||
Strings result_part = listFilesWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash));
|
||||
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
|
||||
}
|
||||
}
|
||||
@ -143,7 +145,7 @@ StorageFile::StorageFile(
|
||||
poco_path = Poco::Path(db_dir_path, poco_path);
|
||||
|
||||
const std::string path = poco_path.absolute().toString();
|
||||
paths = LSWithRegexpMatching("/", path);
|
||||
paths = listFilesWithRegexpMatching("/", path);
|
||||
for (const auto & cur_path : paths)
|
||||
checkCreationIsAllowed(context_global, db_dir_path, cur_path);
|
||||
is_db_table = false;
|
||||
@ -282,7 +284,7 @@ public:
|
||||
else
|
||||
{
|
||||
if (storage.paths.size() != 1)
|
||||
throw Exception("Table '" + storage.table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
write_buf = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT);
|
||||
}
|
||||
|
||||
|
@ -171,6 +171,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c
|
||||
if (re2::RE2::FullMatch(file_name, matcher))
|
||||
{
|
||||
Strings result_part = LSWithRegexpMatching(full_path + "/", fs, suffix_with_globs.substr(next_slash));
|
||||
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
|
||||
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
|
||||
}
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ namespace
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: sample_block(sample_block_)
|
||||
{
|
||||
auto minimum_upload_part_size = context.getConfigRef().getUInt64("s3_minimum_upload_part_size", 512'000'000);
|
||||
auto minimum_upload_part_size = context.getConfigRef().getUInt64("s3.minimum_upload_part_size", 512 * 1024 * 1024);
|
||||
write_buf = std::make_unique<WriteBufferFromS3>(uri, minimum_upload_part_size, timeouts);
|
||||
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
|
||||
}
|
||||
@ -119,14 +119,36 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams StorageS3::read(const Names & column_names,
|
||||
StorageS3::StorageS3(
|
||||
const Poco::URI & uri_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_)
|
||||
: IStorage(columns_)
|
||||
, uri(uri_)
|
||||
, context_global(context_)
|
||||
, format_name(format_name_)
|
||||
, database_name(database_name_)
|
||||
, table_name(table_name_)
|
||||
{
|
||||
setColumns(columns_);
|
||||
setConstraints(constraints_);
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams StorageS3::read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t max_block_size,
|
||||
unsigned /*num_streams*/)
|
||||
{
|
||||
BlockInputStreamPtr block_input = std::make_shared<StorageS3BlockInputStream>(uri,
|
||||
BlockInputStreamPtr block_input = std::make_shared<StorageS3BlockInputStream>(
|
||||
uri,
|
||||
format_name,
|
||||
getName(),
|
||||
getHeaderBlock(column_names),
|
||||
@ -171,7 +193,7 @@ void registerStorageS3(StorageFactory & factory)
|
||||
|
||||
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageS3::create(uri, args.database_name, args.table_name, format_name, args.columns, args.context);
|
||||
return StorageS3::create(uri, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/**
|
||||
@ -15,22 +16,14 @@ namespace DB
|
||||
class StorageS3 : public ext::shared_ptr_helper<StorageS3>, public IStorage
|
||||
{
|
||||
public:
|
||||
StorageS3(const Poco::URI & uri_,
|
||||
StorageS3(
|
||||
const Poco::URI & uri_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_
|
||||
)
|
||||
: IStorage(columns_)
|
||||
, uri(uri_)
|
||||
, context_global(context_)
|
||||
, format_name(format_name_)
|
||||
, database_name(database_name_)
|
||||
, table_name(table_name_)
|
||||
{
|
||||
setColumns(columns_);
|
||||
}
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_);
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -47,7 +40,8 @@ public:
|
||||
return table_name;
|
||||
}
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
@ -58,11 +52,10 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
protected:
|
||||
private:
|
||||
Poco::URI uri;
|
||||
const Context & context_global;
|
||||
|
||||
private:
|
||||
String format_name;
|
||||
String database_name;
|
||||
String table_name;
|
||||
|
@ -54,8 +54,7 @@ namespace
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: name(name_)
|
||||
{
|
||||
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, method, callback, timeouts);
|
||||
|
||||
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects);
|
||||
reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
|
||||
}
|
||||
|
||||
|
@ -2,10 +2,12 @@
|
||||
const char * auto_contributors[] {
|
||||
"0xflotus",
|
||||
"821008736@qq.com",
|
||||
"Akazz",
|
||||
"Alberto",
|
||||
"Aleksandra (Ася)",
|
||||
"Alex Bocharov",
|
||||
"Alex Krash",
|
||||
"Alex Ryndin",
|
||||
"Alex Zatelepin",
|
||||
"Alexander Avdonkin",
|
||||
"Alexander Ermolaev",
|
||||
@ -18,9 +20,11 @@ const char * auto_contributors[] {
|
||||
"Alexander Lukin",
|
||||
"Alexander Makarov",
|
||||
"Alexander Marshalov",
|
||||
"Alexander Mezhov",
|
||||
"Alexander Millin",
|
||||
"Alexander Mochalin",
|
||||
"Alexander Prudaev",
|
||||
"Alexander Rodin",
|
||||
"Alexander Sapin",
|
||||
"Alexander Tokmakov",
|
||||
"Alexander Tretiakov",
|
||||
@ -69,6 +73,7 @@ const char * auto_contributors[] {
|
||||
"Bakhtiyor Ruziev",
|
||||
"BanyRule",
|
||||
"BayoNet",
|
||||
"Big Elephant",
|
||||
"BlahGeek",
|
||||
"Bogdan",
|
||||
"Bogdan Voronin",
|
||||
@ -96,6 +101,8 @@ const char * auto_contributors[] {
|
||||
"Dmitry Petukhov",
|
||||
"Dmitry Rubashkin",
|
||||
"Dmitry S..ky / skype: dvska-at-skype",
|
||||
"Doge",
|
||||
"Eldar Zaitov",
|
||||
"Elghazal Ahmed",
|
||||
"Emmanuel Donin de Rosière",
|
||||
"Eric",
|
||||
@ -105,9 +112,11 @@ const char * auto_contributors[] {
|
||||
"Evgeniy Gatov",
|
||||
"Evgeniy Udodov",
|
||||
"Evgeny Konkov",
|
||||
"Fabian Stäber",
|
||||
"Fadi Hadzh",
|
||||
"FeehanG",
|
||||
"Flowyi",
|
||||
"Francisco Barón",
|
||||
"Fruit of Eden",
|
||||
"Gary Dotzler",
|
||||
"George",
|
||||
@ -121,7 +130,10 @@ const char * auto_contributors[] {
|
||||
"Hiroaki Nakamura",
|
||||
"Igor",
|
||||
"Igor Hatarist",
|
||||
"Igor Mineev",
|
||||
"Igor Strykhar",
|
||||
"Igr",
|
||||
"Igr Mineev",
|
||||
"Ildar Musin",
|
||||
"Ildus Kurbangaliev",
|
||||
"Ilya",
|
||||
@ -235,6 +247,8 @@ const char * auto_contributors[] {
|
||||
"Pawel Rog",
|
||||
"Persiyanov Dmitriy Andreevich",
|
||||
"Quid37",
|
||||
"Rafael David Tinoco",
|
||||
"Ramazan Polat",
|
||||
"Ravengg",
|
||||
"Reto Kromer",
|
||||
"Roman Lipovsky",
|
||||
@ -246,6 +260,7 @@ const char * auto_contributors[] {
|
||||
"SaltTan",
|
||||
"Samuel Chou",
|
||||
"Serge Rider",
|
||||
"Sergei Bocharov",
|
||||
"Sergei Semin",
|
||||
"Sergei Tsetlin (rekub)",
|
||||
"Sergey Elantsev",
|
||||
@ -263,6 +278,7 @@ const char * auto_contributors[] {
|
||||
"Stanislav Pavlovichev",
|
||||
"Stas Pavlovichev",
|
||||
"Stefan Thies",
|
||||
"Stepan Herold",
|
||||
"Stupnikov Andrey",
|
||||
"SuperBot",
|
||||
"Sébastien Launay",
|
||||
@ -271,6 +287,7 @@ const char * auto_contributors[] {
|
||||
"The-Alchemist",
|
||||
"Tobias Adamson",
|
||||
"Tsarkova Anastasia",
|
||||
"VDimir",
|
||||
"Vadim",
|
||||
"Vadim Plakhtinskiy",
|
||||
"Vadim Skipin",
|
||||
@ -284,6 +301,7 @@ const char * auto_contributors[] {
|
||||
"Victor Tarnavsky",
|
||||
"Vitaliy Karnienko",
|
||||
"Vitaliy Lyudvichenko",
|
||||
"Vitaliy Zakaznikov",
|
||||
"Vitaly Baranov",
|
||||
"Vitaly Samigullin",
|
||||
"Vivien Maisonneuve",
|
||||
@ -296,6 +314,7 @@ const char * auto_contributors[] {
|
||||
"Vladislav Smirnov",
|
||||
"Vojtech Splichal",
|
||||
"Vsevolod Orlov",
|
||||
"Vxider",
|
||||
"Vyacheslav Alipov",
|
||||
"Weiqing Xu",
|
||||
"William Shallum",
|
||||
@ -312,9 +331,11 @@ const char * auto_contributors[] {
|
||||
"abdrakhmanov",
|
||||
"abyss7",
|
||||
"achulkov2",
|
||||
"akazz",
|
||||
"akonyaev",
|
||||
"akuzm",
|
||||
"alesapin",
|
||||
"alex-zaitsev",
|
||||
"alexander kozhikhov",
|
||||
"alexey-milovidov",
|
||||
"andrewsg",
|
||||
@ -336,9 +357,13 @@ const char * auto_contributors[] {
|
||||
"chertus",
|
||||
"coraxster",
|
||||
"daoready",
|
||||
"dasmfm",
|
||||
"davydovska",
|
||||
"decaseal",
|
||||
"dependabot[bot]",
|
||||
"dimarub2000",
|
||||
"dmitrii",
|
||||
"dmitriiut",
|
||||
"dmitry kuzmin",
|
||||
"eejoin",
|
||||
"egatov",
|
||||
@ -346,6 +371,7 @@ const char * auto_contributors[] {
|
||||
"ezhaka",
|
||||
"f1yegor",
|
||||
"felixoid",
|
||||
"fenglv",
|
||||
"fessmage",
|
||||
"filimonov",
|
||||
"flow",
|
||||
@ -363,8 +389,10 @@ const char * auto_contributors[] {
|
||||
"javi",
|
||||
"javi santana",
|
||||
"kmeaw",
|
||||
"kreuzerkrieg",
|
||||
"ks1322",
|
||||
"kshvakov",
|
||||
"l",
|
||||
"leozhang",
|
||||
"levushkin aleksej",
|
||||
"levysh",
|
||||
@ -375,10 +403,13 @@ const char * auto_contributors[] {
|
||||
"lomberts",
|
||||
"luc1ph3r",
|
||||
"maiha",
|
||||
"malkfilipp",
|
||||
"maqroll",
|
||||
"maxkuzn",
|
||||
"mf5137",
|
||||
"mfridental",
|
||||
"miha-g",
|
||||
"millb",
|
||||
"morty",
|
||||
"moscas",
|
||||
"never lee",
|
||||
@ -388,7 +419,9 @@ const char * auto_contributors[] {
|
||||
"ogorbacheva",
|
||||
"olegkv",
|
||||
"orantius",
|
||||
"palasonicq",
|
||||
"peshkurov",
|
||||
"philip.han",
|
||||
"proller",
|
||||
"pyos",
|
||||
"qianlixiang",
|
||||
@ -399,6 +432,8 @@ const char * auto_contributors[] {
|
||||
"santaux",
|
||||
"sdk2",
|
||||
"serebrserg",
|
||||
"sev7e0",
|
||||
"sfod",
|
||||
"shangshujie",
|
||||
"shedx",
|
||||
"simon-says",
|
||||
@ -408,6 +443,7 @@ const char * auto_contributors[] {
|
||||
"sundyli",
|
||||
"svladykin",
|
||||
"tai",
|
||||
"tavplubix",
|
||||
"topvisor",
|
||||
"unknown",
|
||||
"urgordeadbeef",
|
||||
@ -427,4 +463,6 @@ const char * auto_contributors[] {
|
||||
"张健",
|
||||
"张风啸",
|
||||
"谢磊",
|
||||
"黄朝晖",
|
||||
"박현우",
|
||||
nullptr};
|
||||
|
@ -5,15 +5,17 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
StoragePtr TableFunctionS3::getStorage(
|
||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const
|
||||
{
|
||||
Poco::URI uri(source);
|
||||
return StorageS3::create(uri, getDatabaseName(), table_name, format, columns, global_context);
|
||||
return StorageS3::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context);
|
||||
}
|
||||
|
||||
void registerTableFunctionS3(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionS3>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,12 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* s3(source, format, structure) - creates a temporary storage for a file in S3
|
||||
*/
|
||||
class TableFunctionS3 : public ITableFunctionFileLike
|
||||
@ -20,6 +21,11 @@ public:
|
||||
|
||||
private:
|
||||
StoragePtr getStorage(
|
||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override;
|
||||
const String & source,
|
||||
const String & format,
|
||||
const ColumnsDescription & columns,
|
||||
Context & global_context,
|
||||
const std::string & table_name) const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,7 @@
|
||||
|
||||
Если аккаунта нет - зарегистрируйтесь на https://github.com/. Создайте ssh ключи, если их нет, и загрузите публичные ключи на GitHub. Это потребуется для отправки изменений. Для работы с GitHub можно использовать такие же ssh ключи, как и для работы с другими ssh серверами - скорее всего, они уже у вас есть.
|
||||
|
||||
Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/yandex/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий.
|
||||
Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/ClickHouse/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий.
|
||||
|
||||
Для работы с git репозиториями, установите `git`.
|
||||
|
||||
@ -61,7 +61,7 @@ and the repository exists.
|
||||
|
||||
Вы также можете клонировать репозиторий по протоколу https:
|
||||
```
|
||||
git clone https://github.com/yandex/ClickHouse.git
|
||||
git clone https://github.com/ClickHouse/ClickHouse.git
|
||||
```
|
||||
Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`.
|
||||
|
||||
@ -228,7 +228,7 @@ sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-fil
|
||||
|
||||
Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/
|
||||
|
||||
Список задач: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md
|
||||
Список задач: https://github.com/ClickHouse/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md
|
||||
|
||||
|
||||
# Тестовые данные
|
||||
|
@ -134,7 +134,7 @@ Geohash - способ преобразования географических
|
||||
|
||||
Энтропию следует считать по гистограмме. Пример расчёта гистограммы смотрите в реализации функции `quantileExact`.
|
||||
|
||||
https://github.com/yandex/ClickHouse/issues/3266
|
||||
https://github.com/ClickHouse/ClickHouse/issues/3266
|
||||
|
||||
## Функции создания и обновления состояния агрегатной функции по одному кортежу аргументов.
|
||||
|
||||
@ -152,7 +152,7 @@ https://github.com/yandex/ClickHouse/issues/3266
|
||||
|
||||
## Корректное сравнение Date и DateTime.
|
||||
|
||||
https://github.com/yandex/ClickHouse/issues/2011
|
||||
https://github.com/ClickHouse/ClickHouse/issues/2011
|
||||
|
||||
Нужно сравнивать Date и DateTime так, как будто Date расширено до DateTime на начало суток в том же часовом поясе.
|
||||
|
||||
|
@ -9,7 +9,7 @@ cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'])
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
|
||||
|
||||
#test reproducing issue https://github.com/yandex/ClickHouse/issues/3162
|
||||
#test reproducing issue https://github.com/ClickHouse/ClickHouse/issues/3162
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
|
45
dbms/tests/integration/test_redirect_url_storage/test.py
Normal file
45
dbms/tests/integration/test_redirect_url_storage/test.py
Normal file
@ -0,0 +1,45 @@
|
||||
import pytest
|
||||
from helpers.hdfs_api import HDFSApi
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', with_zookeeper=False, with_hdfs=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_url_without_redirect(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n")
|
||||
assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n"
|
||||
|
||||
# access datanode port directly
|
||||
node1.query("create table WebHDFSStorage (id UInt32, name String, weight Float64) ENGINE = URL('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV')")
|
||||
assert node1.query("select * from WebHDFSStorage") == "1\tMark\t72.53\n"
|
||||
|
||||
def test_url_with_redirect_not_allowed(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n")
|
||||
assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n"
|
||||
|
||||
# access proxy port without allowing redirects
|
||||
node1.query("create table WebHDFSStorageWithoutRedirect (id UInt32, name String, weight Float64) ENGINE = URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV')")
|
||||
with pytest.raises(Exception):
|
||||
assert node1.query("select * from WebHDFSStorageWithoutRedirect") == "1\tMark\t72.53\n"
|
||||
|
||||
def test_url_with_redirect_allowed(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n")
|
||||
assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n"
|
||||
|
||||
# access proxy port with allowing redirects
|
||||
# http://localhost:50070/webhdfs/v1/b?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0
|
||||
node1.query("create table WebHDFSStorageWithRedirect (id UInt32, name String, weight Float64) ENGINE = URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV')")
|
||||
assert node1.query("SET max_http_get_redirects=1; select * from WebHDFSStorageWithRedirect") == "1\tMark\t72.53\n"
|
@ -29,7 +29,6 @@ def started_cluster():
|
||||
|
||||
def test_read_write_storage(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n")
|
||||
|
||||
node1.query("create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')")
|
||||
node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)")
|
||||
@ -39,19 +38,40 @@ def test_read_write_storage(started_cluster):
|
||||
def test_read_write_storage_with_globs(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
|
||||
for i in ["1", "2", "3"]:
|
||||
hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n")
|
||||
assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n"
|
||||
|
||||
node1.query("create table HDFSStorageWithRange (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1..5}', 'TSV')")
|
||||
node1.query("create table HDFSStorageWithEnum (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1,2,3,4,5}', 'TSV')")
|
||||
node1.query("create table HDFSStorageWithQuestionMark (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage?', 'TSV')")
|
||||
node1.query("create table HDFSStorageWithAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage*', 'TSV')")
|
||||
|
||||
assert node1.query("select count(*) from HDFSStorageWithRange") == '3\n'
|
||||
assert node1.query("select count(*) from HDFSStorageWithEnum") == '3\n'
|
||||
assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == '3\n'
|
||||
assert node1.query("select count(*) from HDFSStorageWithAsterisk") == '3\n'
|
||||
for i in ["1", "2", "3"]:
|
||||
hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n")
|
||||
assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n"
|
||||
|
||||
assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n"
|
||||
assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n"
|
||||
assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n"
|
||||
assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n"
|
||||
|
||||
try:
|
||||
node1.query("insert into HDFSStorageWithEnum values (1, 'NEW', 4.2)")
|
||||
assert False, "Exception have to be thrown"
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert "in readonly mode" in str(ex)
|
||||
|
||||
try:
|
||||
node1.query("insert into HDFSStorageWithQuestionMark values (1, 'NEW', 4.2)")
|
||||
assert False, "Exception have to be thrown"
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert "in readonly mode" in str(ex)
|
||||
|
||||
try:
|
||||
node1.query("insert into HDFSStorageWithAsterisk values (1, 'NEW', 4.2)")
|
||||
assert False, "Exception have to be thrown"
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert "in readonly mode" in str(ex)
|
||||
|
||||
def test_read_write_table(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
@ -78,18 +98,18 @@ def test_bad_hdfs_uri(started_cluster):
|
||||
node1.query("create table BadStorage1 (id UInt32, name String, weight Float64) ENGINE = HDFS('hads:hgsdfs100500:9000/other_storage', 'TSV')")
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert 'Illegal HDFS URI' in str(ex)
|
||||
assert "Illegal HDFS URI" in str(ex)
|
||||
try:
|
||||
node1.query("create table BadStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs100500:9000/other_storage', 'TSV')")
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert 'Unable to create builder to connect to HDFS' in str(ex)
|
||||
assert "Unable to create builder to connect to HDFS" in str(ex)
|
||||
|
||||
try:
|
||||
node1.query("create table BadStorage3 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/<>', 'TSV')")
|
||||
except Exception as ex:
|
||||
print ex
|
||||
assert 'Unable to open HDFS file' in str(ex)
|
||||
assert "Unable to open HDFS file" in str(ex)
|
||||
|
||||
def test_globs_in_read_table(started_cluster):
|
||||
hdfs_api = HDFSApi("root")
|
||||
|
@ -1,3 +1,5 @@
|
||||
<yandex>
|
||||
<s3_minimum_upload_part_size>1000000</s3_minimum_upload_part_size>
|
||||
<s3>
|
||||
<minimum_upload_part_size>1000000</minimum_upload_part_size>
|
||||
</s3>
|
||||
</yandex>
|
||||
|
@ -17,5 +17,5 @@
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : materialize(['a', 'b', 'c']))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : materialize(['a', 'b', 'c']))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(rand() % 2 ? materialize(['', '']) : emptyArrayString())</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(rand() % 2 ? materialize(['https://github.com/yandex/ClickHouse/pull/1070', 'https://www.google.ru/search?newwindow=1&site=&source=hp&q=zookeeper+wire+protocol+exists&oq=zookeeper+wire+protocol+exists&gs_l=psy-ab.3...330.6300.0.6687.33.28.0.0.0.0.386.4838.0j5j9j5.19.0....0...1.1.64.psy-ab..14.17.4448.0..0j35i39k1j0i131k1j0i22i30k1j0i19k1j33i21k1.r_3uFoNOrSU']) : emptyArrayString())</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(rand() % 2 ? materialize(['https://github.com/ClickHouse/ClickHouse/pull/1070', 'https://www.google.ru/search?newwindow=1&site=&source=hp&q=zookeeper+wire+protocol+exists&oq=zookeeper+wire+protocol+exists&gs_l=psy-ab.3...330.6300.0.6687.33.28.0.0.0.0.386.4838.0j5j9j5.19.0....0...1.1.64.psy-ab..14.17.4448.0..0j35i39k1j0i131k1j0i22i30k1j0i19k1j33i21k1.r_3uFoNOrSU']) : emptyArrayString())</query>
|
||||
</test>
|
||||
|
@ -3,7 +3,7 @@
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
# https://github.com/yandex/ClickHouse/issues/1300
|
||||
# https://github.com/ClickHouse/ClickHouse/issues/1300
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser";
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser_test";
|
||||
|
@ -1,5 +1,5 @@
|
||||
|
||||
-- https://github.com/yandex/ClickHouse/issues/1059
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/1059
|
||||
|
||||
DROP TABLE IF EXISTS union1;
|
||||
DROP TABLE IF EXISTS union2;
|
||||
|
@ -38,7 +38,7 @@ select s.*, j.* from (select * from X) as s full join (select * from Y) as j on
|
||||
--select X.*, Y.* from X full join Y on (X.id + 1) = (Y.id + 1) order by id;
|
||||
|
||||
select 'self inner';
|
||||
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id;
|
||||
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
|
||||
select 'self inner nullable';
|
||||
select X.*, s.* from X inner join (select * from X) as s on X.x_b = s.x_b order by X.id;
|
||||
select 'self inner nullable vs not nullable';
|
||||
@ -48,7 +48,7 @@ select 'self inner nullable vs not nullable 2';
|
||||
select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
|
||||
|
||||
select 'self left';
|
||||
select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id;
|
||||
select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
|
||||
select 'self left nullable';
|
||||
select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id;
|
||||
select 'self left nullable vs not nullable';
|
||||
@ -58,7 +58,7 @@ select 'self left nullable vs not nullable 2';
|
||||
select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
|
||||
|
||||
select 'self right';
|
||||
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id;
|
||||
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
|
||||
select 'self right nullable';
|
||||
select X.*, s.* from X right join (select * from X) as s on X.x_b = s.x_b order by X.id;
|
||||
select 'self right nullable vs not nullable';
|
||||
@ -67,7 +67,7 @@ select X.*, s.* from X right join (select * from X) as s on X.id = s.x_b order b
|
||||
--select Y.*, s.* from Y right join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
|
||||
|
||||
select 'self full';
|
||||
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id;
|
||||
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
|
||||
select 'self full nullable';
|
||||
select X.*, s.* from X full join (select * from X) as s on X.x_b = s.x_b order by X.id;
|
||||
select 'self full nullable vs not nullable';
|
||||
|
@ -12,7 +12,9 @@ insert into y values (1,1);
|
||||
|
||||
select t.a, s.b, s.a, s.b, y.a, y.b from t
|
||||
left join s on (t.a = s.a and t.b = s.b)
|
||||
left join y on (y.a = s.a and y.b = s.b) format Vertical;
|
||||
left join y on (y.a = s.a and y.b = s.b)
|
||||
order by t.a
|
||||
format Vertical;
|
||||
|
||||
select t.a, s.b, s.a, s.b, y.a, y.b from t
|
||||
left join s on (t.a = s.a and s.b = t.b)
|
||||
|
@ -37,21 +37,30 @@ INSERT INTO t3_00849 values (1,1), (1, Null);
|
||||
INSERT INTO t4_00849 values (1,1), (1, Null);
|
||||
|
||||
SELECT 'SELECT * FROM t1, t2';
|
||||
SELECT * FROM t1_00849, t2_00849;
|
||||
SELECT * FROM t1_00849, t2_00849
|
||||
ORDER BY t1_00849.a, t2_00849.b;
|
||||
SELECT 'SELECT * FROM t1, t2 WHERE t1.a = t2.a';
|
||||
SELECT * FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a;
|
||||
SELECT * FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a
|
||||
ORDER BY t1_00849.a, t2_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b';
|
||||
SELECT t1_00849.a, t2_00849.b FROM t1_00849, t2_00849 WHERE t1_00849.b = t2_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a';
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a;
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849
|
||||
WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a
|
||||
ORDER BY t2_00849.b, t3_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b';
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a';
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a;
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849
|
||||
WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a
|
||||
ORDER BY t2_00849.b, t3_00849.b, t4_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b';
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b;
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849
|
||||
WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b;
|
||||
SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a';
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a;
|
||||
SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849
|
||||
WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a
|
||||
ORDER BY t2_00849.b, t3_00849.b, t4_00849.b;
|
||||
|
||||
DROP TABLE t1_00849;
|
||||
DROP TABLE t2_00849;
|
||||
|
12
dbms/tests/queries/0_stateless/01013_hex_float.reference
Normal file
12
dbms/tests/queries/0_stateless/01013_hex_float.reference
Normal file
@ -0,0 +1,12 @@
|
||||
000000000000F03F
|
||||
0000000000405940
|
||||
00C84E676DC1AB43
|
||||
2342920CA19CC73B
|
||||
7DC39425AD49B254
|
||||
2C616D8C9DF0423F
|
||||
3BDF4F8D97FE5EC0
|
||||
0A57C742
|
||||
00004843
|
||||
00004943
|
||||
0000000000406940
|
||||
0000000000606940
|
10
dbms/tests/queries/0_stateless/01013_hex_float.sql
Normal file
10
dbms/tests/queries/0_stateless/01013_hex_float.sql
Normal file
@ -0,0 +1,10 @@
|
||||
SELECT hex(1.0);
|
||||
SELECT hex(101.);
|
||||
SELECT hex(1e+18);
|
||||
SELECT hex(1e-20);
|
||||
SELECT hex(1e+100);
|
||||
SELECT hex(0.000578);
|
||||
SELECT hex(-123.978);
|
||||
SELECT hex(toFloat32(99.67));
|
||||
SELECT hex(toFloat32(number)) FROM numbers(200, 2);
|
||||
SELECT hex(toFloat64(number)) FROM numbers(202, 2);
|
@ -0,0 +1,38 @@
|
||||
abcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabc
|
||||
sdfggsdfgg
|
||||
xywq
|
||||
|
||||
abcabcabcabcabcabcabcabcabcabcabcabc
|
||||
sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg
|
||||
xywqxywqxywqxywq
|
||||
plkfplkfplkfplkfplkf
|
||||
abcabcabcabc
|
||||
sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg
|
||||
xywqxywqxywqxywqxywqxywqxywqxywqxywq
|
||||
plkfplkfplkfplkfplkfplkfplkf
|
||||
abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc
|
||||
sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg
|
||||
xywqxywqxywqxywqxywq
|
||||
plkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkfplkf
|
||||
abcabcabcabcabcabcabcabcabcabc
|
||||
sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg
|
||||
xywqxywqxywqxywqxywqxywqxywqxywqxywqxywq
|
||||
plkfplkfplkfplkfplkfplkfplkfplkfplkfplkf
|
||||
abcabcabc
|
||||
abcabc
|
||||
abc
|
||||
|
||||
abcabcabcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabc
|
||||
abcabcabcabcabc
|
||||
abcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc
|
||||
abcabcabcabcabc
|
||||
abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc
|
||||
Hello, world! Hello, world! Hello, world!
|
26
dbms/tests/queries/0_stateless/01013_repeat_function.sql
Normal file
26
dbms/tests/queries/0_stateless/01013_repeat_function.sql
Normal file
@ -0,0 +1,26 @@
|
||||
SELECT repeat('abc', 10);
|
||||
DROP TABLE IF EXISTS defaults;
|
||||
CREATE TABLE defaults
|
||||
(
|
||||
strings String,
|
||||
u8 UInt8,
|
||||
u16 UInt16,
|
||||
u32 UInt32,
|
||||
u64 UInt64
|
||||
)ENGINE = Memory();
|
||||
|
||||
INSERT INTO defaults values ('abc', 3, 12, 4, 56) ('sdfgg', 2, 10, 21, 200) ('xywq', 1, 4, 9, 5) ('plkf', 0, 5, 7,77);
|
||||
|
||||
SELECT repeat(strings, u8) FROM defaults;
|
||||
SELECT repeat(strings, u16) FROM defaults;
|
||||
SELECT repeat(strings, u32) from defaults;
|
||||
SELECT repeat(strings, u64) FROM defaults;
|
||||
SELECT repeat(strings, 10) FROM defaults;
|
||||
SELECT repeat('abc', u8) FROM defaults;
|
||||
SELECT repeat('abc', u16) FROM defaults;
|
||||
SELECT repeat('abc', u32) FROM defaults;
|
||||
SELECT repeat('abc', u64) FROM defaults;
|
||||
|
||||
SELECT repeat('Hello, world! ', 3);
|
||||
|
||||
DROP TABLE defaults;
|
@ -0,0 +1,13 @@
|
||||
1000000
|
||||
0
|
||||
|
||||
1
|
||||
22
|
||||
333
|
||||
4444
|
||||
55555
|
||||
666666
|
||||
7777777
|
||||
88888888
|
||||
999999999
|
||||
10101010101010101010
|
@ -0,0 +1,6 @@
|
||||
SELECT length(repeat('x', 1000000));
|
||||
SELECT length(repeat('', 1000000));
|
||||
SELECT length(repeat('x', 1000001)); -- { serverError 131 }
|
||||
SET max_memory_usage = 100000000;
|
||||
SELECT length(repeat(repeat('Hello, world!', 1000000), 10)); -- { serverError 241 }
|
||||
SELECT repeat(toString(number), number) FROM system.numbers LIMIT 11;
|
4
debian/changelog
vendored
4
debian/changelog
vendored
@ -1,5 +1,5 @@
|
||||
clickhouse (19.15.1.1) unstable; urgency=low
|
||||
clickhouse (19.16.1.1) unstable; urgency=low
|
||||
|
||||
* Modified source code
|
||||
|
||||
-- clickhouse-release <clickhouse-release@yandex-team.ru> Fri, 06 Sep 2019 17:58:30 +0300
|
||||
-- clickhouse-release <clickhouse-release@yandex-team.ru> Tue, 24 Sep 2019 14:47:28 +0300
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.15.1.*
|
||||
ARG version=19.16.1.*
|
||||
|
||||
RUN apt-get update \
|
||||
&& apt-get install --yes --no-install-recommends \
|
||||
|
@ -4,4 +4,4 @@ For more information see [ClickHouse Server Docker Image](https://hub.docker.com
|
||||
|
||||
## License
|
||||
|
||||
View [license information](https://github.com/yandex/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
|
@ -12,5 +12,6 @@
|
||||
"docker/test/stateless_with_coverage": "yandex/clickhouse-stateless-test-with-coverage",
|
||||
"docker/test/unit": "yandex/clickhouse-unit-test",
|
||||
"docker/test/stress": "yandex/clickhouse-stress-test",
|
||||
"docker/test/split_build_smoke_test": "yandex/clickhouse-split-build-smoke-test",
|
||||
"dbms/tests/integration/image": "yandex/clickhouse-integration-tests-runner"
|
||||
}
|
||||
|
@ -59,7 +59,7 @@ RUN apt-get update -y \
|
||||
gdb \
|
||||
rename \
|
||||
wget
|
||||
|
||||
|
||||
# Build and install tools for cross-linking to Darwin
|
||||
|
||||
ENV CC=clang-8
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.15.1.*
|
||||
ARG version=19.16.1.*
|
||||
ARG gosu_ver=1.10
|
||||
|
||||
RUN apt-get update \
|
||||
|
@ -59,4 +59,4 @@ EOSQL
|
||||
|
||||
## License
|
||||
|
||||
View [license information](https://github.com/yandex/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.15.1.*
|
||||
ARG version=19.16.1.*
|
||||
|
||||
RUN apt-get update && \
|
||||
apt-get install -y apt-transport-https dirmngr && \
|
||||
|
@ -2,4 +2,4 @@
|
||||
|
||||
## License
|
||||
|
||||
View [license information](https://github.com/yandex/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image.
|
||||
|
6
docker/test/split_build_smoke_test/Dockerfile
Normal file
6
docker/test/split_build_smoke_test/Dockerfile
Normal file
@ -0,0 +1,6 @@
|
||||
# docker build -t yandex/clickhouse-split-build-smoke-test .
|
||||
FROM yandex/clickhouse-binary-builder
|
||||
|
||||
COPY run.sh /run.sh
|
||||
|
||||
CMD /run.sh
|
17
docker/test/split_build_smoke_test/run.sh
Executable file
17
docker/test/split_build_smoke_test/run.sh
Executable file
@ -0,0 +1,17 @@
|
||||
#!/bin/bash
|
||||
|
||||
set -x
|
||||
|
||||
install_and_run_server() {
|
||||
mkdir /unpacked
|
||||
tar -xzf /package_folder/shared_build.tgz -C /unpacked --strip 1
|
||||
LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-server --config /unpacked/config/config.xml >/var/log/clickhouse-server/stderr.log 2>&1 &
|
||||
sleep 5
|
||||
}
|
||||
|
||||
run_client() {
|
||||
LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-client --query "select 'OK'" 2>/var/log/clickhouse-server/clientstderr.log || echo "FAIL"
|
||||
}
|
||||
|
||||
install_and_run_server
|
||||
run_client
|
@ -8,42 +8,34 @@ Array of `T`-type items.
|
||||
|
||||
You can use a function to create an array:
|
||||
|
||||
```
|
||||
```sql
|
||||
array(T)
|
||||
```
|
||||
|
||||
You can also use square brackets.
|
||||
|
||||
```
|
||||
```sql
|
||||
[]
|
||||
```
|
||||
|
||||
Example of creating an array:
|
||||
|
||||
```sql
|
||||
SELECT array(1, 2) AS x, toTypeName(x)
|
||||
```
|
||||
:) SELECT array(1, 2) AS x, toTypeName(x)
|
||||
|
||||
SELECT
|
||||
[1, 2] AS x,
|
||||
toTypeName(x)
|
||||
|
||||
```text
|
||||
┌─x─────┬─toTypeName(array(1, 2))─┐
|
||||
│ [1,2] │ Array(UInt8) │
|
||||
└───────┴─────────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.002 sec.
|
||||
|
||||
:) SELECT [1, 2] AS x, toTypeName(x)
|
||||
|
||||
SELECT
|
||||
[1, 2] AS x,
|
||||
toTypeName(x)
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT [1, 2] AS x, toTypeName(x)
|
||||
```
|
||||
```text
|
||||
┌─x─────┬─toTypeName([1, 2])─┐
|
||||
│ [1,2] │ Array(UInt8) │
|
||||
└───────┴────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.002 sec.
|
||||
```
|
||||
|
||||
## Working with data types
|
||||
@ -54,31 +46,23 @@ If ClickHouse couldn't determine the data type, it will generate an exception. F
|
||||
|
||||
Examples of automatic data type detection:
|
||||
|
||||
```
|
||||
:) SELECT array(1, 2, NULL) AS x, toTypeName(x)
|
||||
|
||||
SELECT
|
||||
[1, 2, NULL] AS x,
|
||||
toTypeName(x)
|
||||
|
||||
```sql
|
||||
SELECT array(1, 2, NULL) AS x, toTypeName(x)
|
||||
```
|
||||
```text
|
||||
┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐
|
||||
│ [1,2,NULL] │ Array(Nullable(UInt8)) │
|
||||
└────────────┴───────────────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.002 sec.
|
||||
```
|
||||
|
||||
If you try to create an array of incompatible data types, ClickHouse throws an exception:
|
||||
|
||||
```sql
|
||||
SELECT array(1, 'a')
|
||||
```
|
||||
:) SELECT array(1, 'a')
|
||||
|
||||
SELECT [1, 'a']
|
||||
|
||||
```text
|
||||
Received exception from server (version 1.1.54388):
|
||||
Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not.
|
||||
|
||||
0 rows in set. Elapsed: 0.246 sec.
|
||||
```
|
||||
|
||||
|
||||
|
@ -51,36 +51,36 @@ Some functions on Decimal return result as Float64 (for example, var or stddev).
|
||||
|
||||
During calculations on Decimal, integer overflows might happen. Excessive digits in fraction are discarded (not rounded). Excessive digits in integer part will lead to exception.
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT toDecimal32(2, 4) AS x, x / 3
|
||||
```
|
||||
```
|
||||
```text
|
||||
┌──────x─┬─divide(toDecimal32(2, 4), 3)─┐
|
||||
│ 2.0000 │ 0.6666 │
|
||||
└────────┴──────────────────────────────┘
|
||||
```
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT toDecimal32(4.2, 8) AS x, x * x
|
||||
```
|
||||
```
|
||||
```text
|
||||
DB::Exception: Scale is out of bounds.
|
||||
```
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT toDecimal32(4.2, 8) AS x, 6 * x
|
||||
```
|
||||
```
|
||||
```text
|
||||
DB::Exception: Decimal math overflow.
|
||||
```
|
||||
|
||||
Overflow checks lead to operations slowdown. If it is known that overflows are not possible, it makes sense to disable checks using `decimal_check_overflow` setting. When checks are disabled and overflow happens, the result will be incorrect:
|
||||
|
||||
```
|
||||
```sql
|
||||
SET decimal_check_overflow = 0;
|
||||
SELECT toDecimal32(4.2, 8) AS x, 6 * x
|
||||
```
|
||||
```
|
||||
```text
|
||||
┌──────────x─┬─multiply(6, toDecimal32(4.2, 8))─┐
|
||||
│ 4.20000000 │ -17.74967296 │
|
||||
└────────────┴──────────────────────────────────┘
|
||||
@ -88,10 +88,10 @@ SELECT toDecimal32(4.2, 8) AS x, 6 * x
|
||||
|
||||
Overflow checks happen not only on arithmetic operations, but also on value comparison:
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT toDecimal32(1, 8) < 100
|
||||
```
|
||||
```
|
||||
```text
|
||||
DB::Exception: Can't compare.
|
||||
```
|
||||
|
||||
|
@ -4,13 +4,13 @@
|
||||
|
||||
### Basic Usage
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
CREATE TABLE hits (url String, from IPv4) ENGINE = MergeTree() ORDER BY url;
|
||||
|
||||
DESCRIBE TABLE hits;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┐
|
||||
│ url │ String │ │ │ │ │
|
||||
│ from │ IPv4 │ │ │ │ │
|
||||
@ -19,19 +19,19 @@ DESCRIBE TABLE hits;
|
||||
|
||||
OR you can use IPv4 domain as a key:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
CREATE TABLE hits (url String, from IPv4) ENGINE = MergeTree() ORDER BY from;
|
||||
```
|
||||
|
||||
`IPv4` domain supports custom input format as IPv4-strings:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
INSERT INTO hits (url, from) VALUES ('https://wikipedia.org', '116.253.40.133')('https://clickhouse.yandex', '183.247.232.58')('https://clickhouse.yandex/docs/en/', '116.106.34.242');
|
||||
|
||||
SELECT * FROM hits;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─url────────────────────────────────┬───────────from─┐
|
||||
│ https://clickhouse.yandex/docs/en/ │ 116.106.34.242 │
|
||||
│ https://wikipedia.org │ 116.253.40.133 │
|
||||
@ -41,11 +41,11 @@ SELECT * FROM hits;
|
||||
|
||||
Values are stored in compact binary form:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(from), hex(from) FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(from)─┬─hex(from)─┐
|
||||
│ IPv4 │ B7F7E83A │
|
||||
└──────────────────┴───────────┘
|
||||
@ -54,7 +54,7 @@ SELECT toTypeName(from), hex(from) FROM hits LIMIT 1;
|
||||
Domain values are not implicitly convertible to types other than `UInt32`.
|
||||
If you want to convert `IPv4` value to a string, you have to do that explicitly with `IPv4NumToString()` function:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(s), IPv4NumToString(from) as s FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
@ -66,11 +66,11 @@ SELECT toTypeName(s), IPv4NumToString(from) as s FROM hits LIMIT 1;
|
||||
|
||||
Or cast to a `UInt32` value:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(i), CAST(from as UInt32) as i FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(CAST(from, 'UInt32'))─┬──────────i─┐
|
||||
│ UInt32 │ 3086477370 │
|
||||
└──────────────────────────────────┴────────────┘
|
||||
|
@ -4,13 +4,13 @@
|
||||
|
||||
### Basic Usage
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
CREATE TABLE hits (url String, from IPv6) ENGINE = MergeTree() ORDER BY url;
|
||||
|
||||
DESCRIBE TABLE hits;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┐
|
||||
│ url │ String │ │ │ │ │
|
||||
│ from │ IPv6 │ │ │ │ │
|
||||
@ -19,19 +19,19 @@ DESCRIBE TABLE hits;
|
||||
|
||||
OR you can use `IPv6` domain as a key:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
CREATE TABLE hits (url String, from IPv6) ENGINE = MergeTree() ORDER BY from;
|
||||
```
|
||||
|
||||
`IPv6` domain supports custom input as IPv6-strings:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
INSERT INTO hits (url, from) VALUES ('https://wikipedia.org', '2a02:aa08:e000:3100::2')('https://clickhouse.yandex', '2001:44c8:129:2632:33:0:252:2')('https://clickhouse.yandex/docs/en/', '2a02:e980:1e::1');
|
||||
|
||||
SELECT * FROM hits;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─url────────────────────────────────┬─from──────────────────────────┐
|
||||
│ https://clickhouse.yandex │ 2001:44c8:129:2632:33:0:252:2 │
|
||||
│ https://clickhouse.yandex/docs/en/ │ 2a02:e980:1e::1 │
|
||||
@ -41,11 +41,11 @@ SELECT * FROM hits;
|
||||
|
||||
Values are stored in compact binary form:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(from), hex(from) FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(from)─┬─hex(from)────────────────────────┐
|
||||
│ IPv6 │ 200144C8012926320033000002520002 │
|
||||
└──────────────────┴──────────────────────────────────┘
|
||||
@ -54,11 +54,11 @@ SELECT toTypeName(from), hex(from) FROM hits LIMIT 1;
|
||||
Domain values are not implicitly convertible to types other than `FixedString(16)`.
|
||||
If you want to convert `IPv6` value to a string, you have to do that explicitly with `IPv6NumToString()` function:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(s), IPv6NumToString(from) as s FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(IPv6NumToString(from))─┬─s─────────────────────────────┐
|
||||
│ String │ 2001:44c8:129:2632:33:0:252:2 │
|
||||
└───────────────────────────────────┴───────────────────────────────┘
|
||||
@ -66,11 +66,11 @@ SELECT toTypeName(s), IPv6NumToString(from) as s FROM hits LIMIT 1;
|
||||
|
||||
Or cast to a `FixedString(16)` value:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT toTypeName(i), CAST(from as FixedString(16)) as i FROM hits LIMIT 1;
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(CAST(from, 'FixedString(16)'))─┬─i───────┐
|
||||
│ FixedString(16) │ <20><><EFBFBD> │
|
||||
└───────────────────────────────────────────┴─────────┘
|
||||
|
@ -26,19 +26,15 @@ ENGINE = TinyLog
|
||||
Column `x` can only store values that are listed in the type definition: `'hello'` or `'world'`. If you try to save any other value, ClickHouse will raise an exception. 8-bit size for this `Enum` is chosen automatically.
|
||||
|
||||
```sql
|
||||
:) INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello')
|
||||
|
||||
INSERT INTO t_enum VALUES
|
||||
|
||||
INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello')
|
||||
```
|
||||
```text
|
||||
Ok.
|
||||
|
||||
3 rows in set. Elapsed: 0.002 sec.
|
||||
|
||||
:) insert into t_enum values('a')
|
||||
|
||||
INSERT INTO t_enum VALUES
|
||||
|
||||
|
||||
```
|
||||
```sql
|
||||
INSERT INTO t_enum values('a')
|
||||
```
|
||||
```text
|
||||
Exception on client:
|
||||
Code: 49. DB::Exception: Unknown element 'a' for type Enum('hello' = 1, 'world' = 2)
|
||||
```
|
||||
@ -47,7 +43,8 @@ When you query data from the table, ClickHouse outputs the string values from `E
|
||||
|
||||
```sql
|
||||
SELECT * FROM t_enum
|
||||
|
||||
```
|
||||
```text
|
||||
┌─x─────┐
|
||||
│ hello │
|
||||
│ world │
|
||||
@ -59,7 +56,8 @@ If you need to see the numeric equivalents of the rows, you must cast the `Enum`
|
||||
|
||||
```sql
|
||||
SELECT CAST(x, 'Int8') FROM t_enum
|
||||
|
||||
```
|
||||
```text
|
||||
┌─CAST(x, 'Int8')─┐
|
||||
│ 1 │
|
||||
│ 2 │
|
||||
@ -71,7 +69,8 @@ To create an Enum value in a query, you also need to use `CAST`.
|
||||
|
||||
```sql
|
||||
SELECT toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)'))
|
||||
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)'))─┐
|
||||
│ Enum8('a' = 1, 'b' = 2) │
|
||||
└─────────────────────────────────────────────────────┘
|
||||
@ -85,7 +84,7 @@ Neither the string nor the numeric value in an `Enum` can be [NULL](../query_lan
|
||||
|
||||
An `Enum` can be contained in [Nullable](nullable.md) type. So if you create a table using the query
|
||||
|
||||
```
|
||||
```sql
|
||||
CREATE TABLE t_enum_nullable
|
||||
(
|
||||
x Nullable( Enum8('hello' = 1, 'world' = 2) )
|
||||
@ -95,7 +94,7 @@ ENGINE = TinyLog
|
||||
|
||||
it can store not only `'hello'` and `'world'`, but `NULL`, as well.
|
||||
|
||||
```
|
||||
```sql
|
||||
INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL)
|
||||
```
|
||||
|
||||
|
@ -4,7 +4,7 @@ A fixed-length string of `N` bytes (neither characters nor code points).
|
||||
|
||||
To declare a column of `FixedString` type, use the following syntax:
|
||||
|
||||
```
|
||||
```sql
|
||||
<column_name> FixedString(N)
|
||||
```
|
||||
|
||||
@ -30,7 +30,7 @@ When selecting the data, ClickHouse does not remove the null bytes at the end of
|
||||
|
||||
Let's consider the following table with the single `FixedString(2)` column:
|
||||
|
||||
```
|
||||
```text
|
||||
┌─name──┐
|
||||
│ b │
|
||||
└───────┘
|
||||
@ -38,15 +38,14 @@ Let's consider the following table with the single `FixedString(2)` column:
|
||||
|
||||
The query `SELECT * FROM FixedStringTable WHERE a = 'b'` does not return any data as a result. We should complement the filter pattern with null bytes.
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT * FROM FixedStringTable
|
||||
WHERE a = 'b\0'
|
||||
|
||||
```
|
||||
```text
|
||||
┌─a─┐
|
||||
│ b │
|
||||
└───┘
|
||||
|
||||
1 rows in set. Elapsed: 0.002 sec.
|
||||
```
|
||||
|
||||
This behavior differs from MySQL behavior for the `CHAR` type (where strings are padded with spaces, and the spaces are removed for output).
|
||||
|
@ -13,11 +13,11 @@ We recommend that you store data in integer form whenever possible. For example,
|
||||
|
||||
- Computations with floating-point numbers might produce a rounding error.
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT 1 - 0.9
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌───────minus(1, 0.9)─┐
|
||||
│ 0.09999999999999998 │
|
||||
└─────────────────────┘
|
||||
@ -33,11 +33,11 @@ In contrast to standard SQL, ClickHouse supports the following categories of flo
|
||||
|
||||
- `Inf` – Infinity.
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT 0.5 / 0
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─divide(0.5, 0)─┐
|
||||
│ inf │
|
||||
└────────────────┘
|
||||
@ -45,11 +45,11 @@ SELECT 0.5 / 0
|
||||
|
||||
- `-Inf` – Negative infinity.
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT -0.5 / 0
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─divide(-0.5, 0)─┐
|
||||
│ -inf │
|
||||
└─────────────────┘
|
||||
@ -57,11 +57,11 @@ SELECT -0.5 / 0
|
||||
|
||||
- `NaN` – Not a number.
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT 0 / 0
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─divide(0, 0)─┐
|
||||
│ nan │
|
||||
└──────────────┘
|
||||
|
@ -33,7 +33,7 @@ To insert data, use `INSERT SELECT` with aggregate `-State`- functions.
|
||||
|
||||
**Function examples**
|
||||
|
||||
```
|
||||
```sql
|
||||
uniqState(UserID)
|
||||
quantilesState(0.5, 0.9)(SendTiming)
|
||||
```
|
||||
|
@ -4,7 +4,7 @@ A nested data structure is like a nested table. The parameters of a nested data
|
||||
|
||||
Example:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
CREATE TABLE test.visits
|
||||
(
|
||||
CounterID UInt32,
|
||||
@ -35,7 +35,7 @@ In most cases, when working with a nested data structure, its individual columns
|
||||
|
||||
Example:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT
|
||||
Goals.ID,
|
||||
Goals.EventTime
|
||||
@ -44,7 +44,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5
|
||||
LIMIT 10
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐
|
||||
│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │
|
||||
│ [1073752] │ ['2014-03-17 00:28:25'] │
|
||||
@ -63,7 +63,7 @@ It is easiest to think of a nested data structure as a set of multiple column ar
|
||||
|
||||
The only place where a SELECT query can specify the name of an entire nested data structure instead of individual columns is the ARRAY JOIN clause. For more information, see "ARRAY JOIN clause". Example:
|
||||
|
||||
``` sql
|
||||
```sql
|
||||
SELECT
|
||||
Goal.ID,
|
||||
Goal.EventTime
|
||||
@ -73,7 +73,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5
|
||||
LIMIT 10
|
||||
```
|
||||
|
||||
```
|
||||
```text
|
||||
┌─Goal.ID─┬──────Goal.EventTime─┐
|
||||
│ 1073752 │ 2014-03-17 16:38:10 │
|
||||
│ 591325 │ 2014-03-17 16:38:48 │
|
||||
|
@ -17,39 +17,20 @@ To store `Nullable` type values in table column, ClickHouse uses a separate file
|
||||
|
||||
## Usage example
|
||||
|
||||
```sql
|
||||
CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog
|
||||
```
|
||||
:) CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog
|
||||
|
||||
CREATE TABLE t_null
|
||||
(
|
||||
x Int8,
|
||||
y Nullable(Int8)
|
||||
)
|
||||
ENGINE = TinyLog
|
||||
|
||||
Ok.
|
||||
|
||||
0 rows in set. Elapsed: 0.012 sec.
|
||||
|
||||
:) INSERT INTO t_null VALUES (1, NULL), (2, 3)
|
||||
|
||||
INSERT INTO t_null VALUES
|
||||
|
||||
Ok.
|
||||
|
||||
1 rows in set. Elapsed: 0.007 sec.
|
||||
|
||||
:) SELECT x + y FROM t_null
|
||||
|
||||
SELECT x + y
|
||||
FROM t_null
|
||||
|
||||
```sql
|
||||
INSERT INTO t_null VALUES (1, NULL), (2, 3)
|
||||
```
|
||||
```sql
|
||||
SELECT x + y FROM t_null
|
||||
```
|
||||
```text
|
||||
┌─plus(x, y)─┐
|
||||
│ ᴺᵁᴸᴸ │
|
||||
│ 5 │
|
||||
└────────────┘
|
||||
|
||||
2 rows in set. Elapsed: 0.144 sec.
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/data_types/nullable/) <!--hide-->
|
||||
|
@ -7,16 +7,13 @@ For example, literal [NULL](../../query_language/syntax.md#null-literal) has typ
|
||||
|
||||
The `Nothing` type can also used to denote empty arrays:
|
||||
|
||||
```bash
|
||||
:) SELECT toTypeName(array())
|
||||
|
||||
SELECT toTypeName([])
|
||||
|
||||
```sql
|
||||
SELECT toTypeName(array())
|
||||
```
|
||||
```text
|
||||
┌─toTypeName(array())─┐
|
||||
│ Array(Nothing) │
|
||||
└─────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.062 sec.
|
||||
```
|
||||
|
||||
|
||||
|
@ -11,24 +11,19 @@ Tuples can be the result of a query. In this case, for text formats other than J
|
||||
|
||||
You can use a function to create a tuple:
|
||||
|
||||
```
|
||||
```sql
|
||||
tuple(T1, T2, ...)
|
||||
```
|
||||
|
||||
Example of creating a tuple:
|
||||
|
||||
```sql
|
||||
SELECT tuple(1,'a') AS x, toTypeName(x)
|
||||
```
|
||||
:) SELECT tuple(1,'a') AS x, toTypeName(x)
|
||||
|
||||
SELECT
|
||||
(1, 'a') AS x,
|
||||
toTypeName(x)
|
||||
|
||||
```text
|
||||
┌─x───────┬─toTypeName(tuple(1, 'a'))─┐
|
||||
│ (1,'a') │ Tuple(UInt8, String) │
|
||||
└─────────┴───────────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.021 sec.
|
||||
```
|
||||
|
||||
## Working with data types
|
||||
@ -37,18 +32,13 @@ When creating a tuple on the fly, ClickHouse automatically detects the type of e
|
||||
|
||||
Example of automatic data type detection:
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT tuple(1, NULL) AS x, toTypeName(x)
|
||||
|
||||
SELECT
|
||||
(1, NULL) AS x,
|
||||
toTypeName(x)
|
||||
|
||||
```
|
||||
```text
|
||||
┌─x────────┬─toTypeName(tuple(1, NULL))──────┐
|
||||
│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │
|
||||
└──────────┴─────────────────────────────────┘
|
||||
|
||||
1 rows in set. Elapsed: 0.002 sec.
|
||||
```
|
||||
|
||||
|
||||
|
@ -4,13 +4,13 @@ A universally unique identifier (UUID) is a 16-byte number used to identify reco
|
||||
|
||||
The example of UUID type value is represented below:
|
||||
|
||||
```
|
||||
```text
|
||||
61f0c404-5cb3-11e7-907b-a6006ad3dba0
|
||||
```
|
||||
|
||||
If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero:
|
||||
|
||||
```
|
||||
```text
|
||||
00000000-0000-0000-0000-000000000000
|
||||
```
|
||||
|
||||
@ -24,13 +24,16 @@ To generate the UUID value, ClickHouse provides the [generateUUIDv4](../query_la
|
||||
|
||||
This example demonstrates creating a table with the UUID type column and inserting a value into the table.
|
||||
|
||||
``` sql
|
||||
:) CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog
|
||||
|
||||
:) INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1'
|
||||
|
||||
:) SELECT * FROM t_uuid
|
||||
|
||||
```sql
|
||||
CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog
|
||||
```
|
||||
```sql
|
||||
INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1'
|
||||
```
|
||||
```sql
|
||||
SELECT * FROM t_uuid
|
||||
```
|
||||
```text
|
||||
┌────────────────────────────────────x─┬─y─────────┐
|
||||
│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │
|
||||
└──────────────────────────────────────┴───────────┘
|
||||
@ -40,11 +43,13 @@ This example demonstrates creating a table with the UUID type column and inserti
|
||||
|
||||
In this example, the UUID column value is not specified when inserting a new record.
|
||||
|
||||
``` sql
|
||||
:) INSERT INTO t_uuid (y) VALUES ('Example 2')
|
||||
|
||||
:) SELECT * FROM t_uuid
|
||||
|
||||
```sql
|
||||
INSERT INTO t_uuid (y) VALUES ('Example 2')
|
||||
```
|
||||
```sql
|
||||
SELECT * FROM t_uuid
|
||||
```
|
||||
```text
|
||||
┌────────────────────────────────────x─┬─y─────────┐
|
||||
│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │
|
||||
│ 00000000-0000-0000-0000-000000000000 │ Example 2 │
|
||||
|
@ -55,7 +55,7 @@ All other MySQL data types are converted into [String](../data_types/string.md).
|
||||
|
||||
Table in MySQL:
|
||||
|
||||
```
|
||||
```text
|
||||
mysql> USE test;
|
||||
Database changed
|
||||
|
||||
|
@ -3,21 +3,21 @@
|
||||
## Install Git and Pbuilder
|
||||
|
||||
```bash
|
||||
sudo apt-get update
|
||||
sudo apt-get install git pbuilder debhelper lsb-release fakeroot sudo debian-archive-keyring debian-keyring
|
||||
$ sudo apt-get update
|
||||
$ sudo apt-get install git pbuilder debhelper lsb-release fakeroot sudo debian-archive-keyring debian-keyring
|
||||
```
|
||||
|
||||
## Checkout ClickHouse Sources
|
||||
|
||||
```bash
|
||||
git clone --recursive --branch master https://github.com/ClickHouse/ClickHouse.git
|
||||
cd ClickHouse
|
||||
$ git clone --recursive --branch master https://github.com/ClickHouse/ClickHouse.git
|
||||
$ cd ClickHouse
|
||||
```
|
||||
|
||||
## Run Release Script
|
||||
|
||||
```bash
|
||||
./release
|
||||
$ ./release
|
||||
```
|
||||
|
||||
# How to Build ClickHouse for Development
|
||||
@ -29,13 +29,13 @@ Only x86_64 with SSE 4.2 is supported. Support for AArch64 is experimental.
|
||||
To test for SSE 4.2, do
|
||||
|
||||
```bash
|
||||
grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported"
|
||||
$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported"
|
||||
```
|
||||
|
||||
## Install Git and CMake
|
||||
|
||||
```bash
|
||||
sudo apt-get install git cmake ninja-build
|
||||
$ sudo apt-get install git cmake ninja-build
|
||||
```
|
||||
|
||||
Or cmake3 instead of cmake on older systems.
|
||||
@ -47,10 +47,10 @@ There are several ways to do this.
|
||||
### Install from a PPA Package
|
||||
|
||||
```bash
|
||||
sudo apt-get install software-properties-common
|
||||
sudo apt-add-repository ppa:ubuntu-toolchain-r/test
|
||||
sudo apt-get update
|
||||
sudo apt-get install gcc-9 g++-9
|
||||
$ sudo apt-get install software-properties-common
|
||||
$ sudo apt-add-repository ppa:ubuntu-toolchain-r/test
|
||||
$ sudo apt-get update
|
||||
$ sudo apt-get install gcc-9 g++-9
|
||||
```
|
||||
|
||||
### Install from Sources
|
||||
@ -60,33 +60,35 @@ Look at [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/Click
|
||||
## Use GCC 9 for Builds
|
||||
|
||||
```bash
|
||||
export CC=gcc-9
|
||||
export CXX=g++-9
|
||||
$ export CC=gcc-9
|
||||
$ export CXX=g++-9
|
||||
```
|
||||
|
||||
## Install Required Libraries from Packages
|
||||
|
||||
```bash
|
||||
sudo apt-get install libicu-dev libreadline-dev gperf
|
||||
$ sudo apt-get install libicu-dev libreadline-dev gperf
|
||||
```
|
||||
|
||||
## Checkout ClickHouse Sources
|
||||
|
||||
```bash
|
||||
git clone --recursive git@github.com:ClickHouse/ClickHouse.git
|
||||
# or: git clone --recursive https://github.com/ClickHouse/ClickHouse.git
|
||||
|
||||
cd ClickHouse
|
||||
$ git clone --recursive git@github.com:ClickHouse/ClickHouse.git
|
||||
```
|
||||
or
|
||||
```bash
|
||||
$ git clone --recursive https://github.com/ClickHouse/ClickHouse.git
|
||||
$ cd ClickHouse
|
||||
```
|
||||
|
||||
## Build ClickHouse
|
||||
|
||||
```bash
|
||||
mkdir build
|
||||
cd build
|
||||
cmake ..
|
||||
ninja
|
||||
cd ..
|
||||
$ mkdir build
|
||||
$ cd build
|
||||
$ cmake ..
|
||||
$ ninja
|
||||
$ cd ..
|
||||
```
|
||||
|
||||
To create an executable, run `ninja clickhouse`.
|
||||
|
@ -5,22 +5,25 @@ Build should work on Mac OS X 10.12.
|
||||
## Install Homebrew
|
||||
|
||||
```bash
|
||||
/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
|
||||
$ /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
|
||||
```
|
||||
|
||||
## Install Required Compilers, Tools, and Libraries
|
||||
|
||||
```bash
|
||||
brew install cmake ninja gcc icu4c openssl libtool gettext readline gperf
|
||||
$ brew install cmake ninja gcc icu4c openssl libtool gettext readline gperf
|
||||
```
|
||||
|
||||
## Checkout ClickHouse Sources
|
||||
|
||||
```bash
|
||||
git clone --recursive git@github.com:yandex/ClickHouse.git
|
||||
# or: git clone --recursive https://github.com/yandex/ClickHouse.git
|
||||
$ git clone --recursive git@github.com:yandex/ClickHouse.git
|
||||
```
|
||||
or
|
||||
```bash
|
||||
$ git clone --recursive https://github.com/yandex/ClickHouse.git
|
||||
|
||||
cd ClickHouse
|
||||
$ cd ClickHouse
|
||||
```
|
||||
|
||||
For the latest stable version, switch to the `stable` branch.
|
||||
@ -28,11 +31,11 @@ For the latest stable version, switch to the `stable` branch.
|
||||
## Build ClickHouse
|
||||
|
||||
```bash
|
||||
mkdir build
|
||||
cd build
|
||||
cmake .. -DCMAKE_CXX_COMPILER=`which g++-8` -DCMAKE_C_COMPILER=`which gcc-8`
|
||||
ninja
|
||||
cd ..
|
||||
$ mkdir build
|
||||
$ cd build
|
||||
$ cmake .. -DCMAKE_CXX_COMPILER=`which g++-8` -DCMAKE_C_COMPILER=`which gcc-8`
|
||||
$ ninja
|
||||
$ cd ..
|
||||
```
|
||||
|
||||
## Caveats
|
||||
@ -45,7 +48,7 @@ If you intend to run clickhouse-server, make sure to increase the system's maxfi
|
||||
To do so, create the following file:
|
||||
|
||||
/Library/LaunchDaemons/limit.maxfiles.plist:
|
||||
``` xml
|
||||
```xml
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN"
|
||||
"http://www.apple.com/DTDs/PropertyList-1.0.dtd">
|
||||
@ -70,7 +73,7 @@ To do so, create the following file:
|
||||
```
|
||||
|
||||
Execute the following command:
|
||||
``` bash
|
||||
```bash
|
||||
$ sudo chown root:wheel /Library/LaunchDaemons/limit.maxfiles.plist
|
||||
```
|
||||
|
||||
|
@ -86,21 +86,21 @@ Note that all clickhouse tools (server, client, etc) are just symlinks to a sing
|
||||
Alternatively you can install ClickHouse package: either stable release from Yandex repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo service clickhouse-server start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`.
|
||||
|
||||
When ClickHouse is already installed on your system, you can build a new `clickhouse` binary and replace the existing binary:
|
||||
```
|
||||
sudo service clickhouse-server stop
|
||||
sudo cp ./clickhouse /usr/bin/
|
||||
sudo service clickhouse-server start
|
||||
```bash
|
||||
$ sudo service clickhouse-server stop
|
||||
$ sudo cp ./clickhouse /usr/bin/
|
||||
$ sudo service clickhouse-server start
|
||||
```
|
||||
|
||||
Also you can stop system clickhouse-server and run your own with the same configuration but with logging to terminal:
|
||||
```
|
||||
sudo service clickhouse-server stop
|
||||
sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml
|
||||
```bash
|
||||
$ sudo service clickhouse-server stop
|
||||
$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml
|
||||
```
|
||||
|
||||
Example with gdb:
|
||||
```
|
||||
sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml
|
||||
```bash
|
||||
$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml
|
||||
```
|
||||
|
||||
If the system clickhouse-server is already running and you don't want to stop it, you can change port numbers in your `config.xml` (or override them in a file in `config.d` directory), provide appropriate data path, and run it.
|
||||
@ -112,7 +112,7 @@ If the system clickhouse-server is already running and you don't want to stop it
|
||||
|
||||
Before publishing release as stable we deploy it on testing environment. Testing environment is a cluster that process 1/39 part of [Yandex.Metrica](https://metrica.yandex.com/) data. We share our testing environment with Yandex.Metrica team. ClickHouse is upgraded without downtime on top of existing data. We look at first that data is processed successfully without lagging from realtime, the replication continue to work and there is no issues visible to Yandex.Metrica team. First check can be done in the following way:
|
||||
|
||||
```
|
||||
```sql
|
||||
SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h;
|
||||
```
|
||||
|
||||
@ -126,16 +126,16 @@ After deploying to testing environment we run load testing with queries from pro
|
||||
Make sure you have enabled `query_log` on your production cluster.
|
||||
|
||||
Collect query log for a day or more:
|
||||
```
|
||||
clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv
|
||||
```bash
|
||||
$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv
|
||||
```
|
||||
|
||||
This is a way complicated example. `type = 2` will filter queries that are executed successfully. `query LIKE '%ym:%'` is to select relevant queries from Yandex.Metrica. `is_initial_query` is to select only queries that are initiated by client, not by ClickHouse itself (as parts of distributed query processing).
|
||||
|
||||
`scp` this log to your testing cluster and run it as following:
|
||||
|
||||
```
|
||||
clickhouse benchmark --concurrency 16 < queries.tsv
|
||||
```bash
|
||||
$ clickhouse benchmark --concurrency 16 < queries.tsv
|
||||
```
|
||||
(probably you also want to specify a `--user`)
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user