Merge remote-tracking branch 'upstream/master' into replxx

This commit is contained in:
Ivan Lezhankin 2020-01-20 15:52:45 +03:00
commit 03fdadee04
37 changed files with 543 additions and 210 deletions

View File

@ -352,6 +352,7 @@ include (cmake/find/simdjson.cmake)
include (cmake/find/rapidjson.cmake) include (cmake/find/rapidjson.cmake)
include (cmake/find/fastops.cmake) include (cmake/find/fastops.cmake)
include (cmake/find/orc.cmake) include (cmake/find/orc.cmake)
include (cmake/find/replxx.cmake)
find_contrib_lib(cityhash) find_contrib_lib(cityhash)
find_contrib_lib(farmhash) find_contrib_lib(farmhash)

40
cmake/find/replxx.cmake Normal file
View File

@ -0,0 +1,40 @@
option (ENABLE_REPLXX "Enable replxx support" ${NOT_UNBUNDLED})
if (ENABLE_REPLXX)
option (USE_INTERNAL_REPLXX "Use internal replxx library" ${NOT_UNBUNDLED})
if (USE_INTERNAL_REPLXX AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/replxx/README.md")
message (WARNING "submodule contrib/replxx is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_REPLXX 0)
endif ()
if (NOT USE_INTERNAL_REPLXX)
find_library(LIBRARY_REPLXX NAMES replxx replxx-static)
find_path(INCLUDE_REPLXX replxx.hxx)
add_library(replxx UNKNOWN IMPORTED)
set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX})
target_include_directories(replxx PUBLIC ${INCLUDE_REPLXX})
set(CMAKE_REQUIRED_LIBRARIES replxx)
check_cxx_source_compiles(
"
#include <replxx.hxx>
int main() {
replxx::Replxx rx;
}
"
EXTERNAL_REPLXX_WORKS
)
if (NOT EXTERNAL_REPLXX_WORKS)
message (FATAL_ERROR "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}")
endif ()
endif ()
set(USE_REPLXX 1)
message (STATUS "Using replxx")
else ()
set(USE_REPLXX 0)
endif ()

View File

@ -332,4 +332,6 @@ if (USE_FASTOPS)
add_subdirectory (fastops-cmake) add_subdirectory (fastops-cmake)
endif() endif()
add_subdirectory (replxx-cmake) if (USE_INTERNAL_REPLXX)
add_subdirectory (replxx-cmake)
endif()

View File

@ -1,53 +1,18 @@
option (ENABLE_READLINE "Enable readline support" ${ENABLE_LIBRARIES}) set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx")
if (ENABLE_READLINE) set(SRCS
option (USE_INTERNAL_REPLXX "Use internal replxx library" ${NOT_UNBUNDLED}) ${LIBRARY_DIR}/src/conversion.cxx
${LIBRARY_DIR}/src/escape.cxx
${LIBRARY_DIR}/src/history.cxx
${LIBRARY_DIR}/src/io.cxx
${LIBRARY_DIR}/src/prompt.cxx
${LIBRARY_DIR}/src/replxx.cxx
${LIBRARY_DIR}/src/replxx_impl.cxx
${LIBRARY_DIR}/src/util.cxx
${LIBRARY_DIR}/src/wcwidth.cpp
${LIBRARY_DIR}/src/ConvertUTF.cpp
)
if (USE_INTERNAL_REPLXX) add_library(replxx ${SRCS})
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") target_include_directories(replxx PUBLIC ${LIBRARY_DIR}/include)
target_compile_options(replxx PUBLIC -Wno-documentation)
set(SRCS
${LIBRARY_DIR}/src/conversion.cxx
${LIBRARY_DIR}/src/ConvertUTF.cpp
${LIBRARY_DIR}/src/escape.cxx
${LIBRARY_DIR}/src/history.cxx
${LIBRARY_DIR}/src/io.cxx
${LIBRARY_DIR}/src/prompt.cxx
${LIBRARY_DIR}/src/replxx.cxx
${LIBRARY_DIR}/src/replxx_impl.cxx
${LIBRARY_DIR}/src/util.cxx
${LIBRARY_DIR}/src/wcwidth.cpp
${LIBRARY_DIR}/src/windows.cxx
)
add_library(replxx ${SRCS})
target_include_directories(replxx PUBLIC ${LIBRARY_DIR}/include)
target_compile_options(replxx PUBLIC -Wno-documentation)
else ()
find_library(LIBRARY_REPLXX NAMES replxx replxx-static)
find_path(INCLUDE_REPLXX replxx.hxx)
add_library(replxx UNKNOWN IMPORTED)
set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX})
target_include_directories(replxx PUBLIC ${INCLUDE_REPLXX})
set(CMAKE_REQUIRED_LIBRARIES replxx)
check_cxx_source_compiles(
"
#include <replxx.hxx>
int main() {
replxx::Replxx rx;
}
"
EXTERNAL_REPLXX_WORKS
)
if (NOT EXTERNAL_REPLXX_WORKS)
message (FATAL_ERROR "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}")
endif ()
endif ()
target_compile_definitions(replxx PUBLIC USE_REPLXX)
message (STATUS "Using replxx")
endif ()

View File

@ -25,6 +25,7 @@
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <common/find_symbols.h> #include <common/find_symbols.h>
#include <common/LineReader.h>
#include <Common/ClickHouseRevision.h> #include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/Exception.h> #include <Common/Exception.h>
@ -483,7 +484,7 @@ private:
if (server_revision >= Suggest::MIN_SERVER_REVISION && !config().getBool("disable_suggestion", false)) if (server_revision >= Suggest::MIN_SERVER_REVISION && !config().getBool("disable_suggestion", false))
/// Load suggestion data from the server. /// Load suggestion data from the server.
Suggest::instance()->load(connection_parameters, config().getInt("suggestion_limit")); Suggest::instance().load(connection_parameters, config().getInt("suggestion_limit"));
/// Load command history if present. /// Load command history if present.
if (config().has("history_file")) if (config().has("history_file"))
@ -551,17 +552,6 @@ private:
} }
else else
{ {
/// This is intended for testing purposes.
if (config().getBool("always_load_suggestion_data", false))
{
#ifdef USE_REPLXX
SCOPE_EXIT({ Suggest::instance().finalize(); });
Suggest::instance().load(connection_parameters, config().getInt("suggestion_limit"));
#else
throw Exception("Command line suggestions cannot work without line editing library", ErrorCodes::BAD_ARGUMENTS);
#endif
}
query_id = config().getString("query_id", ""); query_id = config().getString("query_id", "");
nonInteractive(); nonInteractive();
@ -1831,13 +1821,6 @@ public:
server_logs_file = options["server_logs_file"].as<std::string>(); server_logs_file = options["server_logs_file"].as<std::string>();
if (options.count("disable_suggestion")) if (options.count("disable_suggestion"))
config().setBool("disable_suggestion", true); config().setBool("disable_suggestion", true);
if (options.count("always_load_suggestion_data"))
{
if (options.count("disable_suggestion"))
throw Exception("Command line parameters disable_suggestion (-A) and always_load_suggestion_data cannot be specified simultaneously",
ErrorCodes::BAD_ARGUMENTS);
config().setBool("always_load_suggestion_data", true);
}
if (options.count("suggestion_limit")) if (options.count("suggestion_limit"))
config().setInt("suggestion_limit", options["suggestion_limit"].as<int>()); config().setInt("suggestion_limit", options["suggestion_limit"].as<int>());

View File

@ -18,10 +18,10 @@ namespace ErrorCodes
class Suggest : public LineReader::Suggest, boost::noncopyable class Suggest : public LineReader::Suggest, boost::noncopyable
{ {
public: public:
static Suggest * instance() static Suggest & instance()
{ {
static Suggest instance; static Suggest instance;
return &instance; return instance;
} }
void load(const ConnectionParameters & connection_parameters, size_t suggestion_limit); void load(const ConnectionParameters & connection_parameters, size_t suggestion_limit);

View File

@ -79,21 +79,19 @@ void MySQLHandler::run()
if (!connection_context.mysql.max_packet_size) if (!connection_context.mysql.max_packet_size)
connection_context.mysql.max_packet_size = MAX_PACKET_LENGTH; connection_context.mysql.max_packet_size = MAX_PACKET_LENGTH;
/* LOG_TRACE(log, "Capabilities: " << handshake_response.capability_flags LOG_TRACE(log, "Capabilities: " << handshake_response.capability_flags
<< "\nmax_packet_size: " << ", max_packet_size: "
<< handshake_response.max_packet_size << handshake_response.max_packet_size
<< "\ncharacter_set: " << ", character_set: "
<< handshake_response.character_set << static_cast<int>(handshake_response.character_set)
<< "\nuser: " << ", user: "
<< handshake_response.username << handshake_response.username
<< "\nauth_response length: " << ", auth_response length: "
<< handshake_response.auth_response.length() << handshake_response.auth_response.length()
<< "\nauth_response: " << ", database: "
<< handshake_response.auth_response
<< "\ndatabase: "
<< handshake_response.database << handshake_response.database
<< "\nauth_plugin_name: " << ", auth_plugin_name: "
<< handshake_response.auth_plugin_name);*/ << handshake_response.auth_plugin_name);
client_capability_flags = handshake_response.capability_flags; client_capability_flags = handshake_response.capability_flags;
if (!(client_capability_flags & CLIENT_PROTOCOL_41)) if (!(client_capability_flags & CLIENT_PROTOCOL_41))

View File

@ -34,7 +34,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
} }
catch (...) catch (...)
{ {
LOG_INFO(log, "Failed to create SSL context. SSL will be disabled. Error: " << getCurrentExceptionMessage(false)); LOG_TRACE(log, "Failed to create SSL context. SSL will be disabled. Error: " << getCurrentExceptionMessage(false));
ssl_enabled = false; ssl_enabled = false;
} }
#endif #endif
@ -47,7 +47,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, "Failed to read RSA keys. Error: " << getCurrentExceptionMessage(false)); LOG_TRACE(log, "Failed to read RSA key pair from server certificate. Error: " << getCurrentExceptionMessage(false));
generateRSAKeys(); generateRSAKeys();
} }
#endif #endif
@ -104,7 +104,7 @@ void MySQLHandlerFactory::readRSAKeys()
void MySQLHandlerFactory::generateRSAKeys() void MySQLHandlerFactory::generateRSAKeys()
{ {
LOG_INFO(log, "Generating new RSA key."); LOG_TRACE(log, "Generating new RSA key pair.");
public_key.reset(RSA_new()); public_key.reset(RSA_new());
if (!public_key) if (!public_key)
throw Exception("Failed to allocate RSA key. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); throw Exception("Failed to allocate RSA key. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);

View File

@ -1030,6 +1030,7 @@ public:
LOG_TRACE(log, "Authentication method match."); LOG_TRACE(log, "Authentication method match.");
} }
bool sent_public_key = false;
if (auth_response == "\1") if (auth_response == "\1")
{ {
LOG_TRACE(log, "Client requests public key."); LOG_TRACE(log, "Client requests public key.");
@ -1050,6 +1051,7 @@ public:
AuthMoreData data(pem); AuthMoreData data(pem);
packet_sender->sendPacket(data, true); packet_sender->sendPacket(data, true);
sent_public_key = true;
AuthSwitchResponse response; AuthSwitchResponse response;
packet_sender->receivePacket(response); packet_sender->receivePacket(response);
@ -1069,13 +1071,15 @@ public:
*/ */
if (!is_secure_connection && !auth_response->empty() && auth_response != String("\0", 1)) if (!is_secure_connection && !auth_response->empty() && auth_response != String("\0", 1))
{ {
LOG_TRACE(log, "Received nonempty password"); LOG_TRACE(log, "Received nonempty password.");
auto ciphertext = reinterpret_cast<unsigned char *>(auth_response->data()); auto ciphertext = reinterpret_cast<unsigned char *>(auth_response->data());
unsigned char plaintext[RSA_size(&private_key)]; unsigned char plaintext[RSA_size(&private_key)];
int plaintext_size = RSA_private_decrypt(auth_response->size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING); int plaintext_size = RSA_private_decrypt(auth_response->size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING);
if (plaintext_size == -1) if (plaintext_size == -1)
{ {
if (!sent_public_key)
LOG_WARNING(log, "Client could have encrypted password with different public key since it didn't request it from server.");
throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
} }

View File

@ -282,29 +282,32 @@ template <
typename Op, template <typename, size_t> typename OperationApplierImpl, size_t N = 10> typename Op, template <typename, size_t> typename OperationApplierImpl, size_t N = 10>
struct OperationApplier struct OperationApplier
{ {
template <typename Columns, typename ResultColumn> template <typename Columns, typename ResultData>
static void apply(Columns & in, ResultColumn & result) static void apply(Columns & in, ResultData & result_data, bool use_result_data_as_input = false)
{ {
while (in.size() > 1) if (!use_result_data_as_input)
{ doBatchedApply<false>(in, result_data);
doBatchedApply(in, result->getData()); while (in.size() > 0)
in.push_back(result.get()); doBatchedApply<true>(in, result_data);
}
} }
template <typename Columns, typename ResultData> template <bool CarryResult, typename Columns, typename ResultData>
static void NO_INLINE doBatchedApply(Columns & in, ResultData & result_data) static void NO_INLINE doBatchedApply(Columns & in, ResultData & result_data)
{ {
if (N > in.size()) if (N > in.size())
{ {
OperationApplier<Op, OperationApplierImpl, N - 1>::doBatchedApply(in, result_data); OperationApplier<Op, OperationApplierImpl, N - 1>
::template doBatchedApply<CarryResult>(in, result_data);
return; return;
} }
const OperationApplierImpl<Op, N> operationApplierImpl(in); const OperationApplierImpl<Op, N> operationApplierImpl(in);
size_t i = 0; size_t i = 0;
for (auto & res : result_data) for (auto & res : result_data)
res = operationApplierImpl.apply(i++); if constexpr (CarryResult)
res = Op::apply(res, operationApplierImpl.apply(i++));
else
res = operationApplierImpl.apply(i++);
in.erase(in.end() - N, in.end()); in.erase(in.end() - N, in.end());
} }
@ -312,9 +315,9 @@ struct OperationApplier
template < template <
typename Op, template <typename, size_t> typename OperationApplierImpl> typename Op, template <typename, size_t> typename OperationApplierImpl>
struct OperationApplier<Op, OperationApplierImpl, 1> struct OperationApplier<Op, OperationApplierImpl, 0>
{ {
template <typename Columns, typename Result> template <bool, typename Columns, typename Result>
static void NO_INLINE doBatchedApply(Columns &, Result &) static void NO_INLINE doBatchedApply(Columns &, Result &)
{ {
throw Exception( throw Exception(
@ -332,7 +335,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn
const bool has_consts = extractConstColumnsTernary<Op>(arguments, const_3v_value); const bool has_consts = extractConstColumnsTernary<Op>(arguments, const_3v_value);
/// If the constant value uniquely determines the result, return it. /// If the constant value uniquely determines the result, return it.
if (has_consts && (arguments.empty() || (Op::isSaturable() && Op::isSaturatedValue(const_3v_value)))) if (has_consts && (arguments.empty() || Op::isSaturatedValue(const_3v_value)))
{ {
result_info.column = ColumnConst::create( result_info.column = ColumnConst::create(
convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()), convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()),
@ -341,16 +344,10 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn
return; return;
} }
const auto result_column = ColumnUInt8::create(input_rows_count); const auto result_column = has_consts ?
MutableColumnPtr const_column_holder; ColumnUInt8::create(input_rows_count, const_3v_value) : ColumnUInt8::create(input_rows_count);
if (has_consts)
{
const_column_holder =
convertFromTernaryData(UInt8Container(input_rows_count, const_3v_value), const_3v_value == Ternary::Null);
arguments.push_back(const_column_holder.get());
}
OperationApplier<Op, AssociativeGenericApplierImpl>::apply(arguments, result_column); OperationApplier<Op, AssociativeGenericApplierImpl>::apply(arguments, result_column->getData(), has_consts);
result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable()); result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable());
} }
@ -425,19 +422,8 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
if (has_consts && Op::apply(const_val, 0) == 0 && Op::apply(const_val, 1) == 1) if (has_consts && Op::apply(const_val, 0) == 0 && Op::apply(const_val, 1) == 1)
has_consts = false; has_consts = false;
UInt8ColumnPtrs uint8_args; auto col_res = has_consts ?
ColumnUInt8::create(input_rows_count, const_val) : ColumnUInt8::create(input_rows_count);
auto col_res = ColumnUInt8::create();
UInt8Container & vec_res = col_res->getData();
if (has_consts)
{
vec_res.assign(input_rows_count, const_val);
uint8_args.push_back(col_res.get());
}
else
{
vec_res.resize(input_rows_count);
}
/// FastPath detection goes in here /// FastPath detection goes in here
if (arguments.size() == (has_consts ? 1 : 2)) if (arguments.size() == (has_consts ? 1 : 2))
@ -452,7 +438,8 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
} }
/// Convert all columns to UInt8 /// Convert all columns to UInt8
Columns converted_columns; UInt8ColumnPtrs uint8_args;
Columns converted_columns_holder;
for (const IColumn * column : arguments) for (const IColumn * column : arguments)
{ {
if (auto uint8_column = checkAndGetColumn<ColumnUInt8>(column)) if (auto uint8_column = checkAndGetColumn<ColumnUInt8>(column))
@ -462,15 +449,11 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
auto converted_column = ColumnUInt8::create(input_rows_count); auto converted_column = ColumnUInt8::create(input_rows_count);
convertColumnToUInt8(column, converted_column->getData()); convertColumnToUInt8(column, converted_column->getData());
uint8_args.push_back(converted_column.get()); uint8_args.push_back(converted_column.get());
converted_columns.emplace_back(std::move(converted_column)); converted_columns_holder.emplace_back(std::move(converted_column));
} }
} }
OperationApplier<Op, AssociativeApplierImpl>::apply(uint8_args, col_res); OperationApplier<Op, AssociativeApplierImpl>::apply(uint8_args, col_res->getData(), has_consts);
/// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8.
if (uint8_args[0] != col_res.get())
vec_res.assign(uint8_args[0]->getData());
result_info.column = std::move(col_res); result_info.column = std::move(col_res);
} }

View File

@ -83,12 +83,7 @@ struct XorImpl
static inline constexpr bool isSaturable() { return false; } static inline constexpr bool isSaturable() { return false; }
static inline constexpr bool isSaturatedValue(bool) { return false; } static inline constexpr bool isSaturatedValue(bool) { return false; }
/** Considering that CH uses UInt8 for representation of boolean values this function static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return !!a != !!b; }
* returns 255 as "true" but the current implementation of logical functions suggests that
* any nonzero value is "true" as well. Also the current code provides no guarantee
* for "true" to be represented with the value of 1.
*/
static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return (a != b) ? Ternary::True : Ternary::False; }
static inline constexpr bool specialImplementationForNulls() { return false; } static inline constexpr bool specialImplementationForNulls() { return false; }
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER

View File

@ -16,7 +16,12 @@ struct BitCountImpl
/// We count bits in the value representation in memory. For example, we support floats. /// We count bits in the value representation in memory. For example, we support floats.
/// We need to avoid sign-extension when converting signed numbers to larger type. So, uint8_t(-1) has 8 bits. /// We need to avoid sign-extension when converting signed numbers to larger type. So, uint8_t(-1) has 8 bits.
return __builtin_popcountll(ext::bit_cast<unsigned long long>(a)); if constexpr (std::is_same_v<A, UInt64> || std::is_same_v<A, Int64>)
return __builtin_popcountll(a);
if constexpr (std::is_same_v<A, UInt32> || std::is_same_v<A, Int32> || std::is_unsigned_v<A>)
return __builtin_popcount(a);
else
return __builtin_popcountll(ext::bit_cast<unsigned long long>(a));
} }
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER

View File

@ -1,5 +1,6 @@
#include <string> #include <string>
#include <iostream> #include <iostream>
#include <cstring>
#include <ryu/ryu.h> #include <ryu/ryu.h>

View File

@ -633,6 +633,10 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new
if (new_ttl_table_ast) if (new_ttl_table_ast)
{ {
std::vector<TTLEntry> update_move_ttl_entries;
ASTPtr update_ttl_table_ast = nullptr;
TTLEntry update_ttl_table_entry;
bool seen_delete_ttl = false; bool seen_delete_ttl = false;
for (auto ttl_element_ptr : new_ttl_table_ast->children) for (auto ttl_element_ptr : new_ttl_table_ast->children)
{ {
@ -647,8 +651,8 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new
auto new_ttl_table_entry = create_ttl_entry(ttl_element.children[0]); auto new_ttl_table_entry = create_ttl_entry(ttl_element.children[0]);
if (!only_check) if (!only_check)
{ {
ttl_table_ast = ttl_element.children[0]; update_ttl_table_ast = ttl_element.children[0];
ttl_table_entry = new_ttl_table_entry; update_ttl_table_entry = new_ttl_table_entry;
} }
seen_delete_ttl = true; seen_delete_ttl = true;
@ -671,11 +675,18 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new
} }
if (!only_check) if (!only_check)
{ update_move_ttl_entries.emplace_back(std::move(new_ttl_entry));
move_ttl_entries.emplace_back(std::move(new_ttl_entry));
}
} }
} }
if (!only_check)
{
ttl_table_entry = update_ttl_table_entry;
ttl_table_ast = update_ttl_table_ast;
auto move_ttl_entries_lock = std::lock_guard<std::mutex>(move_ttl_entries_mutex);
move_ttl_entries = update_move_ttl_entries;
}
} }
} }
@ -3293,7 +3304,7 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_
ReservationPtr reservation; ReservationPtr reservation;
auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move); auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move);
if (ttl_entry != nullptr) if (ttl_entry)
{ {
SpacePtr destination_ptr = ttl_entry->getDestination(storage_policy); SpacePtr destination_ptr = ttl_entry->getDestination(storage_policy);
if (!destination_ptr) if (!destination_ptr)
@ -3352,27 +3363,28 @@ bool MergeTreeData::TTLEntry::isPartInDestination(const StoragePolicyPtr & polic
return false; return false;
} }
const MergeTreeData::TTLEntry * MergeTreeData::selectTTLEntryForTTLInfos( std::optional<MergeTreeData::TTLEntry> MergeTreeData::selectTTLEntryForTTLInfos(
const MergeTreeDataPart::TTLInfos & ttl_infos, const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const time_t time_of_move) const
{ {
const MergeTreeData::TTLEntry * result = nullptr;
/// Prefer TTL rule which went into action last.
time_t max_max_ttl = 0; time_t max_max_ttl = 0;
std::vector<DB::MergeTreeData::TTLEntry>::const_iterator best_entry_it;
for (const auto & ttl_entry : move_ttl_entries) auto lock = std::lock_guard(move_ttl_entries_mutex);
for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it)
{ {
auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry.result_column); auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column);
/// Prefer TTL rule which went into action last.
if (ttl_info_it != ttl_infos.moves_ttl.end() if (ttl_info_it != ttl_infos.moves_ttl.end()
&& ttl_info_it->second.max <= time_of_move && ttl_info_it->second.max <= time_of_move
&& max_max_ttl <= ttl_info_it->second.max) && max_max_ttl <= ttl_info_it->second.max)
{ {
result = &ttl_entry; best_entry_it = ttl_entry_it;
max_max_ttl = ttl_info_it->second.max; max_max_ttl = ttl_info_it->second.max;
} }
} }
return result; return max_max_ttl ? *best_entry_it : std::optional<MergeTreeData::TTLEntry>();
} }
MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const

View File

@ -737,12 +737,17 @@ public:
bool isPartInDestination(const StoragePolicyPtr & policy, const MergeTreeDataPart & part) const; bool isPartInDestination(const StoragePolicyPtr & policy, const MergeTreeDataPart & part) const;
}; };
const TTLEntry * selectTTLEntryForTTLInfos(const MergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; std::optional<TTLEntry> selectTTLEntryForTTLInfos(const MergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const;
using TTLEntriesByName = std::unordered_map<String, TTLEntry>; using TTLEntriesByName = std::unordered_map<String, TTLEntry>;
TTLEntriesByName column_ttl_entries_by_name; TTLEntriesByName column_ttl_entries_by_name;
TTLEntry ttl_table_entry; TTLEntry ttl_table_entry;
/// This mutex is required for background move operations which do not obtain global locks.
mutable std::mutex move_ttl_entries_mutex;
/// Vector rw operations have to be done under "move_ttl_entries_mutex".
std::vector<TTLEntry> move_ttl_entries; std::vector<TTLEntry> move_ttl_entries;
String sampling_expr_column_name; String sampling_expr_column_name;

View File

@ -127,14 +127,14 @@ bool MergeTreePartsMover::selectPartsForMove(
if (!can_move(part, &reason)) if (!can_move(part, &reason))
continue; continue;
const MergeTreeData::TTLEntry * ttl_entry_ptr = part->storage.selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); auto ttl_entry = part->storage.selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move);
auto to_insert = need_to_move.find(part->disk); auto to_insert = need_to_move.find(part->disk);
ReservationPtr reservation; ReservationPtr reservation;
if (ttl_entry_ptr) if (ttl_entry)
{ {
auto destination = ttl_entry_ptr->getDestination(policy); auto destination = ttl_entry->getDestination(policy);
if (destination && !ttl_entry_ptr->isPartInDestination(policy, *part)) if (destination && !ttl_entry->isPartInDestination(policy, *part))
reservation = part->storage.tryReserveSpace(part->bytes_on_disk, ttl_entry_ptr->getDestination(policy)); reservation = part->storage.tryReserveSpace(part->bytes_on_disk, ttl_entry->getDestination(policy));
} }
if (reservation) /// Found reservation by TTL rule. if (reservation) /// Found reservation by TTL rule.

View File

@ -61,6 +61,7 @@ const char * auto_config_build[]
"USE_HYPERSCAN", "@USE_HYPERSCAN@", "USE_HYPERSCAN", "@USE_HYPERSCAN@",
"USE_SIMDJSON", "@USE_SIMDJSON@", "USE_SIMDJSON", "@USE_SIMDJSON@",
"USE_POCO_REDIS", "@USE_POCO_REDIS@", "USE_POCO_REDIS", "@USE_POCO_REDIS@",
"USE_REPLXX", "@USE_REPLXX@",
nullptr, nullptr nullptr, nullptr
}; };

View File

@ -74,37 +74,37 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter):
{expression} {expression}
SETTINGS storage_policy='{policy}' SETTINGS storage_policy='{policy}'
""".format(expression=x, name=name, engine=engine, policy=policy) """.format(expression=x, name=name, engine=engine, policy=policy)
if alter: if alter:
node1.query(get_command(None, "small_jbod_with_external")) node1.query(get_command(None, "small_jbod_with_external"))
with pytest.raises(QueryRuntimeException): with pytest.raises(QueryRuntimeException):
node1.query(get_command("TTL d1 TO DISK 'unknown'", "small_jbod_with_external")) node1.query(get_command("TTL d1 TO DISK 'unknown'", "small_jbod_with_external"))
node1.query("DROP TABLE IF EXISTS {}".format(name)) node1.query("DROP TABLE IF EXISTS {}".format(name))
if alter: if alter:
node1.query(get_command(None, "small_jbod_with_external")) node1.query(get_command(None, "small_jbod_with_external"))
with pytest.raises(QueryRuntimeException): with pytest.raises(QueryRuntimeException):
node1.query(get_command("TTL d1 TO VOLUME 'unknown'", "small_jbod_with_external")) node1.query(get_command("TTL d1 TO VOLUME 'unknown'", "small_jbod_with_external"))
node1.query("DROP TABLE IF EXISTS {}".format(name)) node1.query("DROP TABLE IF EXISTS {}".format(name))
if alter: if alter:
node1.query(get_command(None, "only_jbod2")) node1.query(get_command(None, "only_jbod2"))
with pytest.raises(QueryRuntimeException): with pytest.raises(QueryRuntimeException):
node1.query(get_command("TTL d1 TO DISK 'jbod1'", "only_jbod2")) node1.query(get_command("TTL d1 TO DISK 'jbod1'", "only_jbod2"))
node1.query("DROP TABLE IF EXISTS {}".format(name)) node1.query("DROP TABLE IF EXISTS {}".format(name))
if alter: if alter:
node1.query(get_command(None, "only_jbod2")) node1.query(get_command(None, "only_jbod2"))
with pytest.raises(QueryRuntimeException): with pytest.raises(QueryRuntimeException):
node1.query(get_command("TTL d1 TO VOLUME 'external'", "only_jbod2")) node1.query(get_command("TTL d1 TO VOLUME 'external'", "only_jbod2"))
finally: finally:
node1.query("DROP TABLE IF EXISTS {}".format(name)) node1.query("DROP TABLE IF EXISTS {}".format(name))
@ -501,13 +501,17 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive):
node1.query("DROP TABLE IF EXISTS {}".format(name)) node1.query("DROP TABLE IF EXISTS {}".format(name))
@pytest.mark.parametrize("name,engine,positive", [ @pytest.mark.parametrize("name,engine,positive,bar", [
("mt_test_moves_after_merges_do_not_work","MergeTree()",0), ("mt_test_moves_after_alter_do_not_work","MergeTree()",0,"DELETE"),
("replicated_mt_test_moves_after_merges_do_not_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')",0), ("replicated_mt_test_moves_after_alter_do_not_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')",0,"DELETE"),
("mt_test_moves_after_merges_work","MergeTree()",1), ("mt_test_moves_after_alter_work","MergeTree()",1,"DELETE"),
("replicated_mt_test_moves_after_merges_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')",1), ("replicated_mt_test_moves_after_alter_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')",1,"DELETE"),
("mt_test_moves_after_alter_do_not_work","MergeTree()",0,"TO DISK 'external'"),
("replicated_mt_test_moves_after_alter_do_not_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')",0,"TO DISK 'external'"),
("mt_test_moves_after_alter_work","MergeTree()",1,"TO DISK 'external'"),
("replicated_mt_test_moves_after_alter_work","ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')",1,"TO DISK 'external'"),
]) ])
def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive): def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, bar):
try: try:
node1.query(""" node1.query("""
CREATE TABLE {name} ( CREATE TABLE {name} (
@ -523,8 +527,8 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive):
node1.query(""" node1.query("""
ALTER TABLE {name} ALTER TABLE {name}
MODIFY TTL MODIFY TTL
d1 + INTERVAL 15 MINUTE d1 + INTERVAL 15 MINUTE {bar}
""".format(name=name)) # That shall disable TTL. """.format(name=name, bar=bar)) # That shall disable TTL.
data = [] # 10MB in total data = [] # 10MB in total
for i in range(10): for i in range(10):
@ -578,14 +582,14 @@ limitations under the License."""
) ENGINE = {engine} ) ENGINE = {engine}
ORDER BY tuple() ORDER BY tuple()
PARTITION BY p1 PARTITION BY p1
TTL d1 + INTERVAL 30 SECOND TO DISK 'jbod2', TTL d1 + INTERVAL 30 SECOND TO DISK 'jbod2',
d1 + INTERVAL 60 SECOND TO VOLUME 'external' d1 + INTERVAL 60 SECOND TO VOLUME 'external'
SETTINGS storage_policy='jbods_with_external', merge_with_ttl_timeout=0 SETTINGS storage_policy='jbods_with_external', merge_with_ttl_timeout=0
""".format(name=name, engine=engine)) """.format(name=name, engine=engine))
node1.query(""" node1.query("""
ALTER TABLE {name} MODIFY ALTER TABLE {name} MODIFY
TTL d1 + INTERVAL 0 SECOND TO DISK 'jbod2', TTL d1 + INTERVAL 0 SECOND TO DISK 'jbod2',
d1 + INTERVAL 5 SECOND TO VOLUME 'external', d1 + INTERVAL 5 SECOND TO VOLUME 'external',
d1 + INTERVAL 10 SECOND DELETE d1 + INTERVAL 10 SECOND DELETE
""".format(name=name)) """.format(name=name))
@ -622,6 +626,96 @@ limitations under the License."""
node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("concurrently_altering_ttl_mt","MergeTree()"),
("concurrently_altering_ttl_replicated_mt","ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')",),
])
def test_concurrent_alter_with_ttl_move(started_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy='jbods_with_external'
""".format(name=name, engine=engine))
values = list({ random.randint(1, 1000000) for _ in range(0, 1000) })
def insert(num):
for i in range(num):
day = random.randint(11, 30)
value = values.pop()
month = '0' + str(random.choice([3, 4]))
node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value))
def alter_move(num):
def produce_alter_move(node, name):
move_type = random.choice(["PART", "PARTITION"])
if move_type == "PART":
for _ in range(10):
try:
parts = node1.query("SELECT name from system.parts where table = '{}' and active = 1".format(name)).strip().split('\n')
break
except QueryRuntimeException:
pass
else:
raise Exception("Cannot select from system.parts")
move_part = random.choice(["'" + part + "'" for part in parts])
else:
move_part = random.choice([201903, 201904])
move_disk = random.choice(["DISK", "VOLUME"])
if move_disk == "DISK":
move_volume = random.choice(["'external'", "'jbod1'", "'jbod2'"])
else:
move_volume = random.choice(["'main'", "'external'"])
try:
node1.query("ALTER TABLE {} MOVE {mt} {mp} TO {md} {mv}".format(
name, mt=move_type, mp=move_part, md=move_disk, mv=move_volume))
except QueryRuntimeException as ex:
pass
for i in range(num):
produce_alter_move(node1, name)
def alter_update(num):
for i in range(num):
node1.query("ALTER TABLE {} UPDATE number = number + 1 WHERE 1".format(name))
def alter_modify_ttl(num):
for i in range(num):
ttls = []
for j in range(random.randint(1, 10)):
what = random.choice(["TO VOLUME 'main'", "TO VOLUME 'external'", "TO DISK 'jbod1'", "TO DISK 'jbod2'", "TO DISK 'external'"])
when = "now()+{}".format(random.randint(-1, 5))
ttls.append("{} {}".format(when, what))
node1.query("ALTER TABLE {} MODIFY TTL {}".format(name, ", ".join(ttls)))
def optimize_table(num):
for i in range(num):
node1.query("OPTIMIZE TABLE {} FINAL".format(name))
p = Pool(15)
tasks = []
for i in range(5):
tasks.append(p.apply_async(insert, (100,)))
tasks.append(p.apply_async(alter_move, (100,)))
tasks.append(p.apply_async(alter_update, (100,)))
tasks.append(p.apply_async(alter_modify_ttl, (100,)))
tasks.append(p.apply_async(optimize_table, (100,)))
for task in tasks:
task.get(timeout=60)
assert node1.query("SELECT 1") == "1\n"
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "500\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,positive", [ @pytest.mark.parametrize("name,positive", [
("test_double_move_while_select_negative", 0), ("test_double_move_while_select_negative", 0),
("test_double_move_while_select_positive", 1), ("test_double_move_while_select_positive", 1),

View File

@ -0,0 +1,56 @@
-- Test simple logic over smaller batch of columns
SELECT
-- x1, x2, x3, x4,
xor(x1, x2, x3, x4) AS xor1,
xor(xor(x1, x2), xor(x3, x4)) AS xor2,
or(x1, x2, x3, x4) AS or1,
or(x1 or x2, x3 or x4) AS or2,
and(x1, x2, x3, x4) AS and1,
and(x1 and x2, x3 and x4) AS and2
FROM (
SELECT
toUInt8(number % 2) AS x1,
toUInt8(number / 2 % 2) AS x2,
toUInt8(number / 4 % 2) AS x3,
toUInt8(number / 8 % 2) AS x4
FROM numbers(16)
)
WHERE
xor1 != xor2 OR (and1 != and2 OR or1 != or2)
;
-- Test simple logic over multiple batches of columns (currently batch spans over 10 columns)
SELECT
-- x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11,
xor(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS xor1,
xor(x1, xor(xor(xor(x2, x3), xor(x4, x5)), xor(xor(x6, x7), xor(x8, xor(x9, xor(x10, x11)))))) AS xor2,
or(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS or1,
or(x1, or(or(or(x2, x3), or(x4, x5)), or(or(x6, x7), or(x8, or(x9, or(x10, x11)))))) AS or2,
and(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS and1,
and(x1, and((x2 and x3) and (x4 and x5), (x6 and x7) and (x8 and (x9 and (x10 and x11))))) AS and2
FROM (
SELECT
toUInt8(number % 2) AS x1,
toUInt8(number / 2 % 2) AS x2,
toUInt8(number / 4 % 2) AS x3,
toUInt8(number / 8 % 2) AS x4,
toUInt8(number / 16 % 2) AS x5,
toUInt8(number / 32 % 2) AS x6,
toUInt8(number / 64 % 2) AS x7,
toUInt8(number / 128 % 2) AS x8,
toUInt8(number / 256 % 2) AS x9,
toUInt8(number / 512 % 2) AS x10,
toUInt8(number / 1024 % 2) AS x11
FROM numbers(2048)
)
WHERE
xor1 != xor2 OR (and1 != and2 OR or1 != or2)
;
SELECT 'OK';

View File

@ -0,0 +1,59 @@
-- Tests codepath for ternary logic
SELECT
-- x1, x2, x3, x4,
xor(x1, x2, x3, x4) AS xor1,
xor(xor(x1, x2), xor(x3, x4)) AS xor2,
or(x1, x2, x3, x4) AS or1,
or(x1 or x2, x3 or x4) AS or2,
and(x1, x2, x3, x4) AS and1,
and(x1 and x2, x3 and x4) AS and2
FROM (
SELECT
nullIf(toUInt8(number % 3), 2) AS x1,
nullIf(toUInt8(number / 3 % 3), 2) AS x2,
nullIf(toUInt8(number / 9 % 3), 2) AS x3,
nullIf(toUInt8(number / 27 % 3), 2) AS x4
FROM numbers(81)
)
WHERE
(xor1 != xor2 OR (xor1 is NULL) != (xor2 is NULL)) OR
(or1 != or2 OR (or1 is NULL) != (or2 is NULL) OR (and1 != and2 OR (and1 is NULL) != (and2 is NULL)))
;
-- Test ternary logic over multiple batches of columns (currently batch spans over 10 columns)
SELECT
-- x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11,
xor(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS xor1,
xor(x1, xor(xor(xor(x2, x3), xor(x4, x5)), xor(xor(x6, x7), xor(x8, xor(x9, xor(x10, x11)))))) AS xor2,
or(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS or1,
or(x1, or(or(or(x2, x3), or(x4, x5)), or(or(x6, x7), or(x8, or(x9, or(x10, x11)))))) AS or2,
and(x1, x2, x3, x4, x5, x6, x7, x8, x9, x10, x11) AS and1,
and(x1, and((x2 and x3) and (x4 and x5), (x6 and x7) and (x8 and (x9 and (x10 and x11))))) AS and2
FROM (
SELECT
nullIf(toUInt8(number % 3), 2) AS x1,
nullIf(toUInt8(number / 3 % 3), 2) AS x2,
nullIf(toUInt8(number / 9 % 3), 2) AS x3,
nullIf(toUInt8(number / 27 % 3), 2) AS x4,
nullIf(toUInt8(number / 81 % 3), 2) AS x5,
nullIf(toUInt8(number / 243 % 3), 2) AS x6,
nullIf(toUInt8(number / 729 % 3), 2) AS x7,
nullIf(toUInt8(number / 2187 % 3), 2) AS x8,
nullIf(toUInt8(number / 6561 % 3), 2) AS x9,
nullIf(toUInt8(number / 19683 % 3), 2) AS x10,
nullIf(toUInt8(number / 59049 % 3), 2) AS x11
FROM numbers(177147)
)
WHERE
(xor1 != xor2 OR (xor1 is NULL) != (xor2 is NULL)) OR
(or1 != or2 OR (or1 is NULL) != (or2 is NULL) OR (and1 != and2 OR (and1 is NULL) != (and2 is NULL)))
;
SELECT 'OK';

View File

@ -0,0 +1 @@
1

View File

@ -0,0 +1 @@
((((((((((((((SELECT((((((((((((((((((((((((((((((((1))))))))))))))))))))))))))))))))))))))))))))))

View File

@ -0,0 +1,35 @@
# MySQL interface {#mysql_interface}
ClickHouse supports MySQL wire protocol. It can be enabled by [mysql_port](../operations/server_settings/settings.md#server_settings-mysql_port) setting in configuration file:
```xml
<mysql_port>9004</mysql_port>
```
Example of connecting using command-line tool mysql:
```bash
$ mysql --protocol tcp -u default -P 9004
```
Output if connection succeeded:
```text
Welcome to the MySQL monitor. Commands end with ; or \g.
Your MySQL connection id is 4
Server version: 20.2.1.1-ClickHouse
Copyright (c) 2000, 2019, Oracle and/or its affiliates. All rights reserved.
Oracle is a registered trademark of Oracle Corporation and/or its
affiliates. Other names may be trademarks of their respective
owners.
Type 'help;' or '\h' for help. Type '\c' to clear the current input statement.
mysql>
```
For compatibility with all MySQL clients, it is recommended to specify user password with [double SHA1](../operations/settings/settings_users.md#password_double_sha1_hex) in configuration file.
If user password is specified using [SHA256](../operations/settings/settings_users.md#password_sha256_hex), some clients won't be able to authenticate (mysqljs and old versions of command-line tool mysql).
Restrictions:
- prepared queries are not supported
- some data types are sent as strings

View File

@ -88,18 +88,6 @@ Features:
## Commercial ## Commercial
### Holistics Software
[Holistics](https://www.holistics.io/) was listed by Gartner's Frontrunners in 2019 as one of the top 2 highest ranked business intelligence tools globally for usability. Holistics is a full-stack data platform and business intelligence tool for setting up your analytics processes, built on SQL.
Features:
- Automated email, Slack and Google Sheet schedules of reports.
- Powerful SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters.
- Embedded analytics of reports and dashboards via iframe.
- Data preparation and ETL capabilities.
- SQL data modeling support for relational mapping of data.
### DataGrip ### DataGrip
[DataGrip](https://www.jetbrains.com/datagrip/) is a database IDE from JetBrains with dedicated support for ClickHouse. It is also embedded into other IntelliJ-based tools: PyCharm, IntelliJ IDEA, GoLand, PhpStorm and others. [DataGrip](https://www.jetbrains.com/datagrip/) is a database IDE from JetBrains with dedicated support for ClickHouse. It is also embedded into other IntelliJ-based tools: PyCharm, IntelliJ IDEA, GoLand, PhpStorm and others.
@ -113,6 +101,18 @@ Features:
- Refactorings. - Refactorings.
- Search and Navigation. - Search and Navigation.
### Holistics Software
[Holistics](https://www.holistics.io/) is a full-stack data platform and business intelligence tool.
Features:
- Automated email, Slack and Google Sheet schedules of reports.
- SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters.
- Embedded analytics of reports and dashboards via iframe.
- Data preparation and ETL capabilities.
- SQL data modeling support for relational mapping of data.
### Looker ### Looker
[Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users [Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users
can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their

View File

@ -709,6 +709,20 @@ Positive integer.
<tcp_port_secure>9440</tcp_port_secure> <tcp_port_secure>9440</tcp_port_secure>
``` ```
## mysql_port {#server_settings-mysql_port}
Port for communicating with clients over MySQL protocol.
**Possible values**
Positive integer.
Example
```xml
<mysql_port>9004</mysql_port>
```
## tmp_path ## tmp_path
Path to temporary data for processing large queries. Path to temporary data for processing large queries.

View File

@ -33,12 +33,14 @@ Structure of the `users` section:
### user_name/password ### user_name/password
Password could be specified in plaintext or in SHA256 (hex format). Password can be specified in plaintext or in SHA256 (hex format).
- To assign a password in plaintext (**not recommended**), place it in a `password` element. - To assign a password in plaintext (**not recommended**), place it in a `password` element.
For example, `<password>qwerty</password>`. The password can be left blank. For example, `<password>qwerty</password>`. The password can be left blank.
<a id="password_sha256_hex"></a>
- To assign a password using its SHA256 hash, place it in a `password_sha256_hex` element. - To assign a password using its SHA256 hash, place it in a `password_sha256_hex` element.
For example, `<password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>`. For example, `<password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>`.
@ -51,6 +53,19 @@ Password could be specified in plaintext or in SHA256 (hex format).
The first line of the result is the password. The second line is the corresponding SHA256 hash. The first line of the result is the password. The second line is the corresponding SHA256 hash.
<a id="password_double_sha1_hex"></a>
- For compatibility with MySQL clients, password can be specified in double SHA1 hash. Place it in `password_double_sha1_hex` element.
For example, `<password_double_sha1_hex>08b4a0f1de6ad37da17359e592c8d74788a83eb0</password_double_sha1_hex>`.
Example of how to generate a password from shell:
```
PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
```
The first line of the result is the password. The second line is the corresponding double SHA1 hash.
### user_name/networks ### user_name/networks

View File

@ -0,0 +1,35 @@
# MySQL-интерфейс {#mysql_interface}
ClickHouse поддерживает взаимодействие по протоколу MySQL. Данная функция включается настройкой [mysql_port](../operations/server_settings/settings.md#server_settings-mysql_port) в конфигурационном файле:
```xml
<mysql_port>9004</mysql_port>
```
Пример подключения с помощью стандартного клиента mysql:
```bash
$ mysql --protocol tcp -u default -P 9004
```
Вывод в случае успешного подключения:
```text
Welcome to the MySQL monitor. Commands end with ; or \g.
Your MySQL connection id is 4
Server version: 20.2.1.1-ClickHouse
Copyright (c) 2000, 2019, Oracle and/or its affiliates. All rights reserved.
Oracle is a registered trademark of Oracle Corporation and/or its
affiliates. Other names may be trademarks of their respective
owners.
Type 'help;' or '\h' for help. Type '\c' to clear the current input statement.
mysql>
```
Для совместимости со всеми клиентами рекомендуется задавать пароль пользователя в конфигурационном файле с помощью двойного хэша [SHA1](../operations/settings/settings_users.md#password_double_sha1_hex).
В случае указания пароля с помощью [SHA256](../operations/settings/settings_users.md#password_sha256_hex) некоторые клиенты не смогут пройти аутентификацию (mysqljs и старые версии стандартного клиента mysql).
Ограничения:
- не поддерживаются подготовленные запросы
- некоторые типы данных отправляются как строки

View File

@ -91,18 +91,6 @@
## Коммерческие ## Коммерческие
### Holistics Software
[Holistics](https://www.holistics.io/) вошёл в топ-2 наиболее удобных инструментов для бизнес-аналитики по рейтингу Gartner's Frontrunners в 2019 году. Holistics — full-stack платформа для обработки данных и инструмент бизнес-аналитики, позволяющий вам построить свои процессы с помощью SQL.
Основные возможности:
- Автоматизированные отчёты на почту, Slack, и Google Sheet.
- Мощный редактор SQL c визуализацией, контролем версий, автодополнением, повторным использованием частей запроса и динамическими фильтрами.
- Встроенные инструменты анализа отчётов и всплывающие (iframe) дашборды.
- Подготовка данных и возможности ETL.
- Моделирование данных с помощью SQL для их реляционного отображения.
### DataGrip ### DataGrip
[DataGrip](https://www.jetbrains.com/datagrip/) — это IDE для баз данных о JetBrains с выделенной поддержкой ClickHouse. Он также встроен в другие инструменты на основе IntelliJ: PyCharm, IntelliJ IDEA, GoLand, PhpStorm и другие. [DataGrip](https://www.jetbrains.com/datagrip/) — это IDE для баз данных о JetBrains с выделенной поддержкой ClickHouse. Он также встроен в другие инструменты на основе IntelliJ: PyCharm, IntelliJ IDEA, GoLand, PhpStorm и другие.
@ -116,4 +104,16 @@
- Рефакторинги. - Рефакторинги.
- Поиск и навигация. - Поиск и навигация.
### Holistics Software
[Holistics](https://www.holistics.io/) — full-stack платформа для обработки данных и бизнес-аналитики.
Основные возможности:
- Автоматизированные отчёты на почту, Slack, и Google Sheet.
- Редактор SQL c визуализацией, контролем версий, автодополнением, повторным использованием частей запроса и динамическими фильтрами.
- Встроенные инструменты анализа отчётов и всплывающие (iframe) дашборды.
- Подготовка данных и возможности ETL.
- Моделирование данных с помощью SQL для их реляционного отображения.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party/gui/) <!--hide--> [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party/gui/) <!--hide-->

View File

@ -661,6 +661,16 @@ TCP порт для защищённого обмена данными с кли
<tcp_port_secure>9440</tcp_port_secure> <tcp_port_secure>9440</tcp_port_secure>
``` ```
## mysql_port {#server_settings-mysql_port}
Порт для взаимодействия с клиентами по протоколу MySQL.
Пример
```xml
<mysql_port>9004</mysql_port>
```
## tmp_path ## tmp_path
Путь ко временным данным для обработки больших запросов. Путь ко временным данным для обработки больших запросов.

View File

@ -39,6 +39,8 @@
Например, `<password>qwerty</password>`. Пароль можно оставить пустым. Например, `<password>qwerty</password>`. Пароль можно оставить пустым.
<a id="password_sha256_hex"></a>
- Чтобы назначить пароль в виде SHA256, поместите хэш в элемент `password_sha256_hex`. - Чтобы назначить пароль в виде SHA256, поместите хэш в элемент `password_sha256_hex`.
Например, `<password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>`. Например, `<password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>`.
@ -51,6 +53,20 @@
Первая строка результата — пароль. Вторая строка — соответствующий ему хэш SHA256. Первая строка результата — пароль. Вторая строка — соответствующий ему хэш SHA256.
<a id="password_double_sha1_hex"></a>
- Для совместимости с клиентами MySQL, пароль можно задать с помощью двойного хэша SHA1, поместив его в элемент `password_double_sha1_hex`.
Например, `<password_double_sha1_hex>08b4a0f1de6ad37da17359e592c8d74788a83eb0</password_double_sha1_hex>`.
Пример создания пароля в командной строке:
```
PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
```
Первая строка результата — пароль. Вторая строка — соответствующий ему двойной хэш SHA1.
### user_name/networks ### user_name/networks
Список сетей, из которых пользователь может подключиться к серверу ClickHouse. Список сетей, из которых пользователь может подключиться к серверу ClickHouse.

View File

@ -25,6 +25,7 @@ nav:
- 'Command-Line Client': 'interfaces/cli.md' - 'Command-Line Client': 'interfaces/cli.md'
- 'Native Interface (TCP)': 'interfaces/tcp.md' - 'Native Interface (TCP)': 'interfaces/tcp.md'
- 'HTTP Interface': 'interfaces/http.md' - 'HTTP Interface': 'interfaces/http.md'
- 'MySQL Interface': 'interfaces/mysql.md'
- 'Input and Output Formats': 'interfaces/formats.md' - 'Input and Output Formats': 'interfaces/formats.md'
- 'JDBC Driver': 'interfaces/jdbc.md' - 'JDBC Driver': 'interfaces/jdbc.md'
- 'ODBC Driver': 'interfaces/odbc.md' - 'ODBC Driver': 'interfaces/odbc.md'

View File

@ -26,6 +26,7 @@ nav:
- 'Клиент командной строки': 'interfaces/cli.md' - 'Клиент командной строки': 'interfaces/cli.md'
- 'Нативный интерфейс (TCP)': 'interfaces/tcp.md' - 'Нативный интерфейс (TCP)': 'interfaces/tcp.md'
- 'HTTP-интерфейс': 'interfaces/http.md' - 'HTTP-интерфейс': 'interfaces/http.md'
- 'MySQL-интерфейс': 'interfaces/mysql.md'
- 'Форматы входных и выходных данных': 'interfaces/formats.md' - 'Форматы входных и выходных данных': 'interfaces/formats.md'
- 'JDBC-драйвер': 'interfaces/jdbc.md' - 'JDBC-драйвер': 'interfaces/jdbc.md'
- 'ODBC-драйвер': 'interfaces/odbc.md' - 'ODBC-драйвер': 'interfaces/odbc.md'

View File

@ -104,7 +104,7 @@ if(CCTZ_LIBRARY)
target_link_libraries(common PRIVATE ${CCTZ_LIBRARY}) target_link_libraries(common PRIVATE ${CCTZ_LIBRARY})
endif() endif()
if (ENABLE_REPLXX) if (USE_REPLXX)
target_link_libraries(common PRIVATE replxx) target_link_libraries(common PRIVATE replxx)
endif () endif ()

View File

@ -3,5 +3,6 @@
// .h autogenerated by cmake ! // .h autogenerated by cmake !
#cmakedefine01 USE_JEMALLOC #cmakedefine01 USE_JEMALLOC
#cmakedefine01 USE_REPLXX
#cmakedefine01 UNBUNDLED #cmakedefine01 UNBUNDLED
#cmakedefine01 WITH_COVERAGE #cmakedefine01 WITH_COVERAGE

View File

@ -27,12 +27,9 @@ ReplxxLineReader::ReplxxLineReader(const Suggest * suggest, const String & histo
return replxx::Replxx::completions_t(range.first, range.second); return replxx::Replxx::completions_t(range.first, range.second);
}; };
if (suggest) rx.set_completion_callback(callback);
{ rx.set_complete_on_empty(false);
rx.set_completion_callback(callback); rx.set_word_break_characters(word_break_characters);
rx.set_complete_on_empty(false);
rx.set_word_break_characters(word_break_characters);
}
} }
ReplxxLineReader::~ReplxxLineReader() ReplxxLineReader::~ReplxxLineReader()