Merge branch 'master' into allow_atomic_database_inside_materialize_mysql

This commit is contained in:
Ivan 2020-12-08 16:45:51 +03:00 committed by GitHub
commit 0ba712a2d3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
561 changed files with 47001 additions and 2633 deletions

11
.gitignore vendored
View File

@ -125,4 +125,15 @@ website/package-lock.json
# Toolchains
/cmake/toolchain/*
# ANTLR extension cache
.antlr
# ANTLR generated files
/src/Parsers/New/*.interp
/src/Parsers/New/*.tokens
/src/Parsers/New/ClickHouseParserBaseVisitor.*
# pytest-profiling
/prof
*.iml

12
.gitmodules vendored
View File

@ -142,9 +142,6 @@
[submodule "contrib/replxx"]
path = contrib/replxx
url = https://github.com/ClickHouse-Extras/replxx.git
[submodule "contrib/ryu"]
path = contrib/ryu
url = https://github.com/ClickHouse-Extras/ryu.git
[submodule "contrib/avro"]
path = contrib/avro
url = https://github.com/ClickHouse-Extras/avro.git
@ -172,6 +169,9 @@
[submodule "contrib/fmtlib"]
path = contrib/fmtlib
url = https://github.com/fmtlib/fmt.git
[submodule "contrib/antlr4-runtime"]
path = contrib/antlr4-runtime
url = https://github.com/ClickHouse-Extras/antlr4-runtime.git
[submodule "contrib/sentry-native"]
path = contrib/sentry-native
url = https://github.com/ClickHouse-Extras/sentry-native.git
@ -197,8 +197,7 @@
url = https://github.com/danlark1/miniselect
[submodule "contrib/rocksdb"]
path = contrib/rocksdb
url = https://github.com/facebook/rocksdb
branch = v6.14.5
url = https://github.com/ClickHouse-Extras/rocksdb.git
[submodule "contrib/xz"]
path = contrib/xz
url = https://github.com/xz-mirror/xz
@ -206,3 +205,6 @@
path = contrib/abseil-cpp
url = https://github.com/ClickHouse-Extras/abseil-cpp.git
branch = lts_2020_02_25
[submodule "contrib/dragonbox"]
path = contrib/dragonbox
url = https://github.com/ClickHouse-Extras/dragonbox.git

View File

@ -257,6 +257,8 @@ if (WITH_COVERAGE AND COMPILER_GCC)
set(WITHOUT_COVERAGE "-fno-profile-arcs -fno-test-coverage")
endif()
set(COMPILER_FLAGS "${COMPILER_FLAGS}")
set (CMAKE_BUILD_COLOR_MAKEFILE ON)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS}")
set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_CXX_FLAGS_ADD}")

View File

@ -16,5 +16,4 @@ ClickHouse® is an open-source column-oriented database management system that a
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [SF Bay Area ClickHouse Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/274498897/) on 2 December 2020.
* [SF Bay Area ClickHouse Virtual Office Hours (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/274273549/) on 20 January 2020.

View File

@ -76,12 +76,6 @@
# define NO_SANITIZE_THREAD
#endif
#if defined __GNUC__ && !defined __clang__
# define OPTIMIZE(x) __attribute__((__optimize__(x)))
#else
# define OPTIMIZE(x)
#endif
/// A macro for suppressing warnings about unused variables or function results.
/// Useful for structured bindings which have no standard way to declare this.
#define UNUSED(...) (void)(__VA_ARGS__)

View File

@ -768,7 +768,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
signal_listener_thread.start(*signal_listener);
#if defined(__ELF__) && !defined(__FreeBSD__)
String build_id_hex = DB::SymbolIndex::instance().getBuildIDHex();
String build_id_hex = DB::SymbolIndex::instance()->getBuildIDHex();
if (build_id_hex.empty())
build_id_info = "no build id";
else

View File

@ -179,7 +179,7 @@ void SentryWriter::onFault(int sig, const std::string & error_message, const Sta
sentry_set_extra("signal_number", sentry_value_new_int32(sig));
#if defined(__ELF__) && !defined(__FreeBSD__)
const String & build_id_hex = DB::SymbolIndex::instance().getBuildIDHex();
const String & build_id_hex = DB::SymbolIndex::instance()->getBuildIDHex();
sentry_set_tag("build_id", build_id_hex.c_str());
#endif

View File

@ -104,6 +104,11 @@ void Connection::connect(const char* db,
if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile_arg))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
/// Enables auto-reconnect.
bool reconnect = true;
if (mysql_options(driver.get(), MYSQL_OPT_RECONNECT, reinterpret_cast<const char *>(&reconnect)))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
/// Specifies particular ssl key and certificate if it needs
if (mysql_ssl_set(driver.get(), ifNotEmpty(ssl_key), ifNotEmpty(ssl_cert), ifNotEmpty(ssl_ca), nullptr, nullptr))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
@ -115,11 +120,6 @@ void Connection::connect(const char* db,
if (mysql_set_character_set(driver.get(), "UTF8"))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
/// Enables auto-reconnect.
bool reconnect = true;
if (mysql_options(driver.get(), MYSQL_OPT_RECONNECT, reinterpret_cast<const char *>(&reconnect)))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
is_connected = true;
}

View File

@ -26,6 +26,7 @@ void Pool::Entry::incrementRefCount()
mysql_thread_init();
}
void Pool::Entry::decrementRefCount()
{
if (!data)
@ -150,28 +151,39 @@ Pool::Entry Pool::tryGet()
initialize();
/// Searching for connection which was established but wasn't used.
for (auto & connection : connections)
/// Try to pick an idle connection from already allocated
for (auto connection_it = connections.cbegin(); connection_it != connections.cend();)
{
if (connection->ref_count == 0)
Connection * connection_ptr = *connection_it;
/// Fixme: There is a race condition here b/c we do not synchronize with Pool::Entry's copy-assignment operator
if (connection_ptr->ref_count == 0)
{
Entry res(connection, this);
return res.tryForceConnected() ? res : Entry();
Entry res(connection_ptr, this);
if (res.tryForceConnected()) /// Tries to reestablish connection as well
return res;
auto & logger = Poco::Util::Application::instance().logger();
logger.information("Idle connection to mysql server cannot be recovered, dropping it.");
/// This one is disconnected, cannot be reestablished and so needs to be disposed of.
connection_it = connections.erase(connection_it);
::delete connection_ptr; /// TODO: Manual memory management is awkward (matches allocConnection() method)
}
else
++connection_it;
}
/// Throws if pool is overflowed.
if (connections.size() >= max_connections)
throw Poco::Exception("mysqlxx::Pool is full");
/// Allocates new connection.
Connection * conn = allocConnection(true);
if (conn)
return Entry(conn, this);
Connection * connection_ptr = allocConnection(true);
if (connection_ptr)
return {connection_ptr, this};
return Entry();
return {};
}
void Pool::removeConnection(Connection* connection)
{
std::lock_guard<std::mutex> lock(mutex);
@ -199,11 +211,9 @@ void Pool::Entry::forceConnected() const
throw Poco::RuntimeException("Tried to access NULL database connection.");
Poco::Util::Application & app = Poco::Util::Application::instance();
if (data->conn.ping())
return;
bool first = true;
do
while (!tryForceConnected())
{
if (first)
first = false;
@ -225,7 +235,26 @@ void Pool::Entry::forceConnected() const
pool->rw_timeout,
pool->enable_local_infile);
}
while (!data->conn.ping());
}
bool Pool::Entry::tryForceConnected() const
{
auto * const mysql_driver = data->conn.getDriver();
const auto prev_connection_id = mysql_thread_id(mysql_driver);
if (data->conn.ping()) /// Attempts to reestablish lost connection
{
const auto current_connection_id = mysql_thread_id(mysql_driver);
if (prev_connection_id != current_connection_id)
{
auto & logger = Poco::Util::Application::instance().logger();
logger.information("Connection to mysql server has been reestablished. Connection id changed: %lu -> %lu",
prev_connection_id, current_connection_id);
}
return true;
}
return false;
}

View File

@ -127,10 +127,7 @@ public:
void forceConnected() const;
/// Connects to database. If connection is failed then returns false.
bool tryForceConnected() const
{
return data->conn.ping();
}
bool tryForceConnected() const;
void incrementRefCount();
void decrementRefCount();

View File

@ -1,2 +1,5 @@
add_executable (mysqlxx_test mysqlxx_test.cpp)
target_link_libraries (mysqlxx_test PRIVATE mysqlxx)
add_executable (mysqlxx_pool_test mysqlxx_pool_test.cpp)
target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx)

View File

@ -0,0 +1,98 @@
#include <mysqlxx/mysqlxx.h>
#include <chrono>
#include <iostream>
#include <sstream>
#include <thread>
namespace
{
mysqlxx::Pool::Entry getWithFailover(mysqlxx::Pool & connections_pool)
{
using namespace std::chrono;
constexpr size_t max_tries = 3;
mysqlxx::Pool::Entry worker_connection;
for (size_t try_no = 1; try_no <= max_tries; ++try_no)
{
try
{
worker_connection = connections_pool.tryGet();
if (!worker_connection.isNull())
{
return worker_connection;
}
}
catch (const Poco::Exception & e)
{
if (e.displayText().find("mysqlxx::Pool is full") != std::string::npos)
{
std::cerr << e.displayText() << std::endl;
}
std::cerr << "Connection to " << connections_pool.getDescription() << " failed: " << e.displayText() << std::endl;
}
std::clog << "Connection to all replicas failed " << try_no << " times" << std::endl;
std::this_thread::sleep_for(1s);
}
std::stringstream message;
message << "Connections to all replicas failed: " << connections_pool.getDescription();
throw Poco::Exception(message.str());
}
}
int main(int, char **)
{
using namespace std::chrono;
const char * remote_mysql = "localhost";
const std::string test_query = "SHOW DATABASES";
mysqlxx::Pool mysql_conn_pool("", remote_mysql, "default", "10203040", 3306);
size_t iteration = 0;
while (++iteration)
{
std::clog << "Iteration: " << iteration << std::endl;
try
{
std::clog << "Acquiring DB connection ...";
mysqlxx::Pool::Entry worker = getWithFailover(mysql_conn_pool);
std::clog << "ok" << std::endl;
std::clog << "Preparing query (5s sleep) ...";
std::this_thread::sleep_for(5s);
mysqlxx::Query query = worker->query();
query << test_query;
std::clog << "ok" << std::endl;
std::clog << "Querying result (5s sleep) ...";
std::this_thread::sleep_for(5s);
mysqlxx::UseQueryResult result = query.use();
std::clog << "ok" << std::endl;
std::clog << "Fetching result data (5s sleep) ...";
std::this_thread::sleep_for(5s);
size_t rows_count = 0;
while (result.fetch())
++rows_count;
std::clog << "ok" << std::endl;
std::clog << "Read " << rows_count << " rows." << std::endl;
}
catch (const Poco::Exception & e)
{
std::cerr << "Iteration FAILED:\n" << e.displayText() << std::endl;
}
std::clog << "====================" << std::endl;
std::this_thread::sleep_for(3s);
}
}

View File

@ -24,7 +24,7 @@ option (WEVERYTHING "Enable -Weverything option with some exceptions." ON)
# Control maximum size of stack frames. It can be important if the code is run in fibers with small stack size.
# Only in release build because debug has too large stack frames.
if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE) AND (NOT CMAKE_CXX_COMPILER_ID MATCHES "AppleClang"))
add_warning(frame-larger-than=32768)
add_warning(frame-larger-than=65536)
endif ()
if (COMPILER_CLANG)

View File

@ -21,6 +21,7 @@ endif()
set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1)
add_subdirectory (antlr4-runtime-cmake)
add_subdirectory (boost-cmake)
add_subdirectory (cctz-cmake)
add_subdirectory (consistent-hashing-sumbur)
@ -34,7 +35,6 @@ add_subdirectory (libmetrohash)
add_subdirectory (lz4-cmake)
add_subdirectory (murmurhash)
add_subdirectory (replxx-cmake)
add_subdirectory (ryu-cmake)
add_subdirectory (unixodbc-cmake)
add_subdirectory (xz)
@ -321,3 +321,5 @@ endif()
if (USE_INTERNAL_ROCKSDB_LIBRARY)
add_subdirectory(rocksdb-cmake)
endif()
add_subdirectory(dragonbox)

1
contrib/antlr4-runtime vendored Submodule

@ -0,0 +1 @@
Subproject commit a2fa7b76e2ee16d2ad955e9214a90bbf79da66fc

View File

@ -0,0 +1,156 @@
set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/antlr4-runtime)
set (SRCS
${LIBRARY_DIR}/ANTLRErrorListener.cpp
${LIBRARY_DIR}/ANTLRErrorStrategy.cpp
${LIBRARY_DIR}/ANTLRFileStream.cpp
${LIBRARY_DIR}/ANTLRInputStream.cpp
${LIBRARY_DIR}/atn/AbstractPredicateTransition.cpp
${LIBRARY_DIR}/atn/ActionTransition.cpp
${LIBRARY_DIR}/atn/AmbiguityInfo.cpp
${LIBRARY_DIR}/atn/ArrayPredictionContext.cpp
${LIBRARY_DIR}/atn/ATN.cpp
${LIBRARY_DIR}/atn/ATNConfig.cpp
${LIBRARY_DIR}/atn/ATNConfigSet.cpp
${LIBRARY_DIR}/atn/ATNDeserializationOptions.cpp
${LIBRARY_DIR}/atn/ATNDeserializer.cpp
${LIBRARY_DIR}/atn/ATNSerializer.cpp
${LIBRARY_DIR}/atn/ATNSimulator.cpp
${LIBRARY_DIR}/atn/ATNState.cpp
${LIBRARY_DIR}/atn/AtomTransition.cpp
${LIBRARY_DIR}/atn/BasicBlockStartState.cpp
${LIBRARY_DIR}/atn/BasicState.cpp
${LIBRARY_DIR}/atn/BlockEndState.cpp
${LIBRARY_DIR}/atn/BlockStartState.cpp
${LIBRARY_DIR}/atn/ContextSensitivityInfo.cpp
${LIBRARY_DIR}/atn/DecisionEventInfo.cpp
${LIBRARY_DIR}/atn/DecisionInfo.cpp
${LIBRARY_DIR}/atn/DecisionState.cpp
${LIBRARY_DIR}/atn/EmptyPredictionContext.cpp
${LIBRARY_DIR}/atn/EpsilonTransition.cpp
${LIBRARY_DIR}/atn/ErrorInfo.cpp
${LIBRARY_DIR}/atn/LexerAction.cpp
${LIBRARY_DIR}/atn/LexerActionExecutor.cpp
${LIBRARY_DIR}/atn/LexerATNConfig.cpp
${LIBRARY_DIR}/atn/LexerATNSimulator.cpp
${LIBRARY_DIR}/atn/LexerChannelAction.cpp
${LIBRARY_DIR}/atn/LexerCustomAction.cpp
${LIBRARY_DIR}/atn/LexerIndexedCustomAction.cpp
${LIBRARY_DIR}/atn/LexerModeAction.cpp
${LIBRARY_DIR}/atn/LexerMoreAction.cpp
${LIBRARY_DIR}/atn/LexerPopModeAction.cpp
${LIBRARY_DIR}/atn/LexerPushModeAction.cpp
${LIBRARY_DIR}/atn/LexerSkipAction.cpp
${LIBRARY_DIR}/atn/LexerTypeAction.cpp
${LIBRARY_DIR}/atn/LL1Analyzer.cpp
${LIBRARY_DIR}/atn/LookaheadEventInfo.cpp
${LIBRARY_DIR}/atn/LoopEndState.cpp
${LIBRARY_DIR}/atn/NotSetTransition.cpp
${LIBRARY_DIR}/atn/OrderedATNConfigSet.cpp
${LIBRARY_DIR}/atn/ParseInfo.cpp
${LIBRARY_DIR}/atn/ParserATNSimulator.cpp
${LIBRARY_DIR}/atn/PlusBlockStartState.cpp
${LIBRARY_DIR}/atn/PlusLoopbackState.cpp
${LIBRARY_DIR}/atn/PrecedencePredicateTransition.cpp
${LIBRARY_DIR}/atn/PredicateEvalInfo.cpp
${LIBRARY_DIR}/atn/PredicateTransition.cpp
${LIBRARY_DIR}/atn/PredictionContext.cpp
${LIBRARY_DIR}/atn/PredictionMode.cpp
${LIBRARY_DIR}/atn/ProfilingATNSimulator.cpp
${LIBRARY_DIR}/atn/RangeTransition.cpp
${LIBRARY_DIR}/atn/RuleStartState.cpp
${LIBRARY_DIR}/atn/RuleStopState.cpp
${LIBRARY_DIR}/atn/RuleTransition.cpp
${LIBRARY_DIR}/atn/SemanticContext.cpp
${LIBRARY_DIR}/atn/SetTransition.cpp
${LIBRARY_DIR}/atn/SingletonPredictionContext.cpp
${LIBRARY_DIR}/atn/StarBlockStartState.cpp
${LIBRARY_DIR}/atn/StarLoopbackState.cpp
${LIBRARY_DIR}/atn/StarLoopEntryState.cpp
${LIBRARY_DIR}/atn/TokensStartState.cpp
${LIBRARY_DIR}/atn/Transition.cpp
${LIBRARY_DIR}/atn/WildcardTransition.cpp
${LIBRARY_DIR}/BailErrorStrategy.cpp
${LIBRARY_DIR}/BaseErrorListener.cpp
${LIBRARY_DIR}/BufferedTokenStream.cpp
${LIBRARY_DIR}/CharStream.cpp
${LIBRARY_DIR}/CommonToken.cpp
${LIBRARY_DIR}/CommonTokenFactory.cpp
${LIBRARY_DIR}/CommonTokenStream.cpp
${LIBRARY_DIR}/ConsoleErrorListener.cpp
${LIBRARY_DIR}/DefaultErrorStrategy.cpp
${LIBRARY_DIR}/dfa/DFA.cpp
${LIBRARY_DIR}/dfa/DFASerializer.cpp
${LIBRARY_DIR}/dfa/DFAState.cpp
${LIBRARY_DIR}/dfa/LexerDFASerializer.cpp
${LIBRARY_DIR}/DiagnosticErrorListener.cpp
${LIBRARY_DIR}/Exceptions.cpp
${LIBRARY_DIR}/FailedPredicateException.cpp
${LIBRARY_DIR}/InputMismatchException.cpp
${LIBRARY_DIR}/InterpreterRuleContext.cpp
${LIBRARY_DIR}/IntStream.cpp
${LIBRARY_DIR}/Lexer.cpp
${LIBRARY_DIR}/LexerInterpreter.cpp
${LIBRARY_DIR}/LexerNoViableAltException.cpp
${LIBRARY_DIR}/ListTokenSource.cpp
${LIBRARY_DIR}/misc/InterpreterDataReader.cpp
${LIBRARY_DIR}/misc/Interval.cpp
${LIBRARY_DIR}/misc/IntervalSet.cpp
${LIBRARY_DIR}/misc/MurmurHash.cpp
${LIBRARY_DIR}/misc/Predicate.cpp
${LIBRARY_DIR}/NoViableAltException.cpp
${LIBRARY_DIR}/Parser.cpp
${LIBRARY_DIR}/ParserInterpreter.cpp
${LIBRARY_DIR}/ParserRuleContext.cpp
${LIBRARY_DIR}/ProxyErrorListener.cpp
${LIBRARY_DIR}/RecognitionException.cpp
${LIBRARY_DIR}/Recognizer.cpp
${LIBRARY_DIR}/RuleContext.cpp
${LIBRARY_DIR}/RuleContextWithAltNum.cpp
${LIBRARY_DIR}/RuntimeMetaData.cpp
${LIBRARY_DIR}/support/Any.cpp
${LIBRARY_DIR}/support/Arrays.cpp
${LIBRARY_DIR}/support/CPPUtils.cpp
${LIBRARY_DIR}/support/guid.cpp
${LIBRARY_DIR}/support/StringUtils.cpp
${LIBRARY_DIR}/Token.cpp
${LIBRARY_DIR}/TokenSource.cpp
${LIBRARY_DIR}/TokenStream.cpp
${LIBRARY_DIR}/TokenStreamRewriter.cpp
${LIBRARY_DIR}/tree/ErrorNode.cpp
${LIBRARY_DIR}/tree/ErrorNodeImpl.cpp
${LIBRARY_DIR}/tree/IterativeParseTreeWalker.cpp
${LIBRARY_DIR}/tree/ParseTree.cpp
${LIBRARY_DIR}/tree/ParseTreeListener.cpp
${LIBRARY_DIR}/tree/ParseTreeVisitor.cpp
${LIBRARY_DIR}/tree/ParseTreeWalker.cpp
${LIBRARY_DIR}/tree/pattern/Chunk.cpp
${LIBRARY_DIR}/tree/pattern/ParseTreeMatch.cpp
${LIBRARY_DIR}/tree/pattern/ParseTreePattern.cpp
${LIBRARY_DIR}/tree/pattern/ParseTreePatternMatcher.cpp
${LIBRARY_DIR}/tree/pattern/RuleTagToken.cpp
${LIBRARY_DIR}/tree/pattern/TagChunk.cpp
${LIBRARY_DIR}/tree/pattern/TextChunk.cpp
${LIBRARY_DIR}/tree/pattern/TokenTagToken.cpp
${LIBRARY_DIR}/tree/TerminalNode.cpp
${LIBRARY_DIR}/tree/TerminalNodeImpl.cpp
${LIBRARY_DIR}/tree/Trees.cpp
${LIBRARY_DIR}/tree/xpath/XPath.cpp
${LIBRARY_DIR}/tree/xpath/XPathElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathLexer.cpp
${LIBRARY_DIR}/tree/xpath/XPathLexerErrorListener.cpp
${LIBRARY_DIR}/tree/xpath/XPathRuleAnywhereElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathRuleElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathTokenAnywhereElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathTokenElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathWildcardAnywhereElement.cpp
${LIBRARY_DIR}/tree/xpath/XPathWildcardElement.cpp
${LIBRARY_DIR}/UnbufferedCharStream.cpp
${LIBRARY_DIR}/UnbufferedTokenStream.cpp
${LIBRARY_DIR}/Vocabulary.cpp
${LIBRARY_DIR}/WritableToken.cpp
)
add_library (antlr4-runtime ${SRCS})
target_include_directories (antlr4-runtime SYSTEM PUBLIC ${LIBRARY_DIR})

2
contrib/boost vendored

@ -1 +1 @@
Subproject commit a04e72c0464f0c31d3384f18f0c0db36a05538e0
Subproject commit a7ceabe4747ecc3309dd3dcd9de4b29660dfd298

1
contrib/dragonbox vendored Submodule

@ -0,0 +1 @@
Subproject commit b2751c65c0592c0239aec3becd53d0ea2fde9329

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit f3d791f6568b99366d089b4479f76a515beb66d5
Subproject commit b5523bb9b4bc4239640cbfec4d734be8b8585639

2
contrib/rocksdb vendored

@ -1 +1 @@
Subproject commit 35d8e36ef1b8e3e0759ca81215f855226a0a54bd
Subproject commit 8b966f0ca298fc1475bd09d9775f32dff0fdce0a

View File

@ -345,6 +345,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_cache.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_garbage.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_meta.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_reader.cc
@ -460,6 +461,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util_debug.cc
${ROCKSDB_SOURCE_DIR}/options/cf_options.cc
${ROCKSDB_SOURCE_DIR}/options/configurable.cc
${ROCKSDB_SOURCE_DIR}/options/customizable.cc
${ROCKSDB_SOURCE_DIR}/options/db_options.cc
${ROCKSDB_SOURCE_DIR}/options/options.cc
${ROCKSDB_SOURCE_DIR}/options/options_helper.cc
@ -583,8 +585,9 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc
${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc
${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_tracker.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point_lock_tracker.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_manager.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_tracker.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_manager.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction_db_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction.cc
@ -592,7 +595,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/transactions/snapshot_checker.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_base.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_db_mutex_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_lock_mgr.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_util.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc

1
contrib/ryu vendored

@ -1 +0,0 @@
Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b

View File

@ -1,10 +0,0 @@
SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/ryu)
add_library(ryu
${LIBRARY_DIR}/ryu/d2fixed.c
${LIBRARY_DIR}/ryu/d2s.c
${LIBRARY_DIR}/ryu/f2s.c
${LIBRARY_DIR}/ryu/generic_128.c
)
target_include_directories(ryu SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}")

View File

@ -94,8 +94,8 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
pid="$!"
# check if clickhouse is ready to accept connections
# will try to send ping clickhouse via http_port (max 12 retries, with 1 sec delay)
if ! wget --spider --quiet --prefer-family=IPv6 --tries=12 --waitretry=1 --retry-connrefused "http://localhost:$HTTP_PORT/ping" ; then
# will try to send ping clickhouse via http_port (max 12 retries by default, with 1 sec delay)
if ! wget --spider --quiet --prefer-family=IPv6 --tries="${CLICKHOUSE_INIT_TIMEOUT:-12}" --waitretry=1 --retry-connrefused "http://localhost:$HTTP_PORT/ping" ; then
echo >&2 'ClickHouse init process failed.'
exit 1
fi

View File

@ -131,12 +131,12 @@ function clone_submodules
cd "$FASTTEST_SOURCE"
SUBMODULES_TO_UPDATE=(
contrib/antlr4-runtime
contrib/boost
contrib/zlib-ng
contrib/libxml2
contrib/poco
contrib/libunwind
contrib/ryu
contrib/fmtlib
contrib/base64
contrib/cctz
@ -154,6 +154,7 @@ function clone_submodules
contrib/croaring
contrib/miniselect
contrib/xz
contrib/dragonbox
)
git submodule sync
@ -317,6 +318,9 @@ function run_tests
01545_system_errors
# Checks system.errors
01563_distributed_query_finish
# nc - command not found
01601_proxy_protocol
)
time clickhouse-test -j 8 --order=random --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"

View File

@ -28,6 +28,7 @@ RUN apt-get update \
libssl-dev \
libcurl4-openssl-dev \
gdb \
software-properties-common \
&& rm -rf \
/var/lib/apt/lists/* \
/var/cache/debconf \
@ -37,6 +38,22 @@ RUN apt-get update \
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add -
RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}"
RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
docker-ce \
&& rm -rf \
/var/lib/apt/lists/* \
/var/cache/debconf \
/tmp/* \
&& apt-get clean
RUN dockerd --version; docker --version
RUN python3 -m pip install \
PyMySQL \
aerospike \
@ -60,28 +77,6 @@ RUN python3 -m pip install \
tzlocal \
urllib3
ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 17.09.1-ce
RUN set -eux; \
\
# this "case" statement is generated via "update.sh"
\
if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \
echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \
exit 1; \
fi; \
\
tar --extract \
--file docker.tgz \
--strip-components 1 \
--directory /usr/local/bin/ \
; \
rm docker.tgz; \
\
dockerd --version; \
docker --version
COPY modprobe.sh /usr/local/bin/modprobe
COPY dockerd-entrypoint.sh /usr/local/bin/
COPY compose/ /compose/

View File

@ -8,6 +8,7 @@ RUN apt-get update -y \
apt-get install --yes --no-install-recommends \
brotli \
expect \
zstd \
lsof \
ncdu \
netcat-openbsd \

View File

@ -8,6 +8,7 @@ RUN apt-get --allow-unauthenticated update -y \
apt-get --allow-unauthenticated install --yes --no-install-recommends \
alien \
brotli \
zstd \
cmake \
devscripts \
expect \

View File

@ -24,6 +24,7 @@ RUN apt-get update -y \
tree \
moreutils \
brotli \
zstd \
gdb \
lsof \
unixodbc \

View File

@ -184,6 +184,10 @@ Sparse indexes allow you to work with a very large number of table rows, because
ClickHouse does not require a unique primary key. You can insert multiple rows with the same primary key.
You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses. To allow this feature, turn on the [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key) setting.
The [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause.
### Selecting the Primary Key {#selecting-the-primary-key}
The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may:
@ -579,6 +583,7 @@ Tags:
- `disk` — a disk within a volume.
- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volumes disks.
- `move_factor` — when the amount of available space gets lower than this factor, data automatically start to move on the next volume if any (by default, 0.1).
- `prefer_not_to_merge` — Disables merging of data parts on this volume. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks.
Cofiguration examples:
@ -607,6 +612,18 @@ Cofiguration examples:
</volumes>
<move_factor>0.2</move_factor>
</moving_from_ssd_to_hdd>
<small_jbod_with_external_no_merges>
<volumes>
<main>
<disk>jbod1</disk>
</main>
<external>
<disk>external</disk>
<prefer_not_to_merge>true</prefer_not_to_merge>
</external>
</volumes>
</small_jbod_with_external_no_merges>
</policies>
...
</storage_configuration>

View File

@ -117,8 +117,8 @@ You can compile packages and install them or use programs without installing pac
Youll need to create a data and metadata folders and `chown` them for the desired user. Their paths can be changed in server config (src/programs/server/config.xml), by default they are:
/opt/clickhouse/data/default/
/opt/clickhouse/metadata/default/
/var/lib/clickhouse/data/default/
/var/lib/clickhouse/metadata/default/
On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources.

View File

@ -48,6 +48,7 @@ toc_title: Adopters
| <a href="https://www.flipkart.com/" class="favicon">Flipkart</a> | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) |
| <a href="https://fun.co/rp" class="favicon">FunCorp</a> | Games | | — | — | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) |
| <a href="https://geniee.co.jp" class="favicon">Geniee</a> | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) |
| <a href="https://www.genotek.ru/" class="favicon">Genotek</a> | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) |
| <a href="https://www.huya.com/" class="favicon">HUYA</a> | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) |
| <a href="https://www.the-ica.com/" class="favicon">ICA</a> | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) |
| <a href="https://www.idealista.com" class="favicon">Idealista</a> | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) |

View File

@ -2364,4 +2364,15 @@ Allows configurable `NULL` representation for [TSV](../../interfaces/formats.md#
Default value: `\N`.
## allow_nullable_key {#allow-nullable-key}
Allows using of the [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) tables.
Possible values:
- 1 — `Nullable`-type expressions are allowed in keys.
- 0 — `Nullable`-type expressions are not allowed in keys.
Default value: `0`.
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->

View File

@ -23,4 +23,44 @@ Please note that `errors_count` is updated once per query to the cluster, but `e
- [distributed_replica_error_cap setting](../../operations/settings/settings.md#settings-distributed_replica_error_cap)
- [distributed_replica_error_half_life setting](../../operations/settings/settings.md#settings-distributed_replica_error_half_life)
**Example**
```sql
:) SELECT * FROM system.clusters LIMIT 2 FORMAT Vertical;
```
```text
Row 1:
──────
cluster: test_cluster
shard_num: 1
shard_weight: 1
replica_num: 1
host_name: clickhouse01
host_address: 172.23.0.11
port: 9000
is_local: 1
user: default
default_database:
errors_count: 0
estimated_recovery_time: 0
Row 2:
──────
cluster: test_cluster
shard_num: 1
shard_weight: 1
replica_num: 2
host_name: clickhouse02
host_address: 172.23.0.12
port: 9000
is_local: 0
user: default
default_database:
errors_count: 0
estimated_recovery_time: 0
2 rows in set. Elapsed: 0.002 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/clusters) <!--hide-->

View File

@ -23,4 +23,50 @@ The `system.columns` table contains the following columns (the column type is sh
- `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression.
- `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name.
**Example**
```sql
:) select * from system.columns LIMIT 2 FORMAT Vertical;
```
```text
Row 1:
──────
database: system
table: aggregate_function_combinators
name: name
type: String
default_kind:
default_expression:
data_compressed_bytes: 0
data_uncompressed_bytes: 0
marks_bytes: 0
comment:
is_in_partition_key: 0
is_in_sorting_key: 0
is_in_primary_key: 0
is_in_sampling_key: 0
compression_codec:
Row 2:
──────
database: system
table: aggregate_function_combinators
name: is_internal
type: UInt8
default_kind:
default_expression:
data_compressed_bytes: 0
data_uncompressed_bytes: 0
marks_bytes: 0
comment:
is_in_partition_key: 0
is_in_sorting_key: 0
is_in_primary_key: 0
is_in_sampling_key: 0
compression_codec:
2 rows in set. Elapsed: 0.002 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/columns) <!--hide-->

View File

@ -11,3 +11,21 @@ Columns:
- `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parameter of disk configuration.
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/disks) <!--hide-->
**Example**
```sql
:) SELECT * FROM system.disks;
```
```text
┌─name────┬─path─────────────────┬───free_space─┬──total_space─┬─keep_free_space─┐
│ default │ /var/lib/clickhouse/ │ 276392587264 │ 490652508160 │ 0 │
└─────────┴──────────────────────┴──────────────┴──────────────┴─────────────────┘
1 rows in set. Elapsed: 0.001 sec.
```

View File

@ -8,3 +8,26 @@ Columns:
- `is_aggregate`(`UInt8`) — Whether the function is aggregate.
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/functions) <!--hide-->
**Example**
```sql
SELECT * FROM system.functions LIMIT 10;
```
```text
┌─name─────────────────────┬─is_aggregate─┬─case_insensitive─┬─alias_to─┐
│ sumburConsistentHash │ 0 │ 0 │ │
│ yandexConsistentHash │ 0 │ 0 │ │
│ demangle │ 0 │ 0 │ │
│ addressToLine │ 0 │ 0 │ │
│ JSONExtractRaw │ 0 │ 0 │ │
│ JSONExtractKeysAndValues │ 0 │ 0 │ │
│ JSONExtract │ 0 │ 0 │ │
│ JSONExtractString │ 0 │ 0 │ │
│ JSONExtractFloat │ 0 │ 0 │ │
│ JSONExtractInt │ 0 │ 0 │ │
└──────────────────────────┴──────────────┴──────────────────┴──────────┘
10 rows in set. Elapsed: 0.002 sec.
```

View File

@ -10,4 +10,45 @@ Columns:
- `type` (String) — Setting type (implementation specific string value).
- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed.
**Example**
```sql
:) SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical;
```
```text
Row 1:
──────
name: index_granularity
value: 8192
changed: 0
description: How many rows correspond to one primary key value.
type: SettingUInt64
Row 2:
──────
name: min_bytes_for_wide_part
value: 0
changed: 0
description: Minimal uncompressed size in bytes to create part in wide format instead of compact
type: SettingUInt64
Row 3:
──────
name: min_rows_for_wide_part
value: 0
changed: 0
description: Minimal number of rows to create part in wide format instead of compact
type: SettingUInt64
Row 4:
──────
name: merge_max_block_size
value: 8192
changed: 0
description: How many rows in blocks should be formed for merge operations.
type: SettingUInt64
4 rows in set. Elapsed: 0.001 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/merge_tree_settings) <!--hide-->

View File

@ -6,4 +6,27 @@ You can use this table for tests, or if you need to do a brute force search.
Reads from this table are not parallelized.
**Example**
```sql
:) SELECT * FROM system.numbers LIMIT 10;
```
```text
┌─number─┐
│ 0 │
│ 1 │
│ 2 │
│ 3 │
│ 4 │
│ 5 │
│ 6 │
│ 7 │
│ 8 │
│ 9 │
└────────┘
10 rows in set. Elapsed: 0.001 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/numbers) <!--hide-->

View File

@ -4,4 +4,27 @@ The same as [system.numbers](../../operations/system-tables/numbers.md) but read
Used for tests.
**Example**
```sql
:) SELECT * FROM system.numbers_mt LIMIT 10;
```
```text
┌─number─┐
│ 0 │
│ 1 │
│ 2 │
│ 3 │
│ 4 │
│ 5 │
│ 6 │
│ 7 │
│ 8 │
│ 9 │
└────────┘
10 rows in set. Elapsed: 0.001 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/numbers_mt) <!--hide-->

View File

@ -6,4 +6,18 @@ This table is used if a `SELECT` query doesnt specify the `FROM` clause.
This is similar to the `DUAL` table found in other DBMSs.
**Example**
```sql
:) SELECT * FROM system.one LIMIT 10;
```
```text
┌─dummy─┐
│ 0 │
└───────┘
1 rows in set. Elapsed: 0.001 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/one) <!--hide-->

View File

@ -14,4 +14,51 @@ Columns:
- `query` (String) The query text. For `INSERT`, it doesnt include the data to insert.
- `query_id` (String) Query ID, if defined.
```sql
:) SELECT * FROM system.processes LIMIT 10 FORMAT Vertical;
```
```text
Row 1:
──────
is_initial_query: 1
user: default
query_id: 35a360fa-3743-441d-8e1f-228c938268da
address: ::ffff:172.23.0.1
port: 47588
initial_user: default
initial_query_id: 35a360fa-3743-441d-8e1f-228c938268da
initial_address: ::ffff:172.23.0.1
initial_port: 47588
interface: 1
os_user: bharatnc
client_hostname: tower
client_name: ClickHouse
client_revision: 54437
client_version_major: 20
client_version_minor: 7
client_version_patch: 2
http_method: 0
http_user_agent:
quota_key:
elapsed: 0.000582537
is_cancelled: 0
read_rows: 0
read_bytes: 0
total_rows_approx: 0
written_rows: 0
written_bytes: 0
memory_usage: 0
peak_memory_usage: 0
query: SELECT * from system.processes LIMIT 10 FORMAT Vertical;
thread_ids: [67]
ProfileEvents.Names: ['Query','SelectQuery','ReadCompressedBytes','CompressedReadBufferBlocks','CompressedReadBufferBytes','IOBufferAllocs','IOBufferAllocBytes','ContextLock','RWLockAcquiredReadLocks']
ProfileEvents.Values: [1,1,36,1,10,1,89,16,1]
Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage']
Settings.Values: ['0','in_order','1','10000000000']
1 rows in set. Elapsed: 0.002 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/processes) <!--hide-->

View File

@ -10,6 +10,7 @@ Columns:
- `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy.
- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit).
- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order.
- `prefer_not_to_merge` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Value of the `prefer_not_to_merge` setting. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks.
If the storage policy contains more then one volume, then information for each volume is stored in the individual row of the table.

View File

@ -52,4 +52,56 @@ This table contains the following columns (the column type is shown in brackets)
The `system.tables` table is used in `SHOW TABLES` query implementation.
```sql
:) SELECT * FROM system.tables LIMIT 2 FORMAT Vertical;
```
```text
Row 1:
──────
database: system
name: aggregate_function_combinators
uuid: 00000000-0000-0000-0000-000000000000
engine: SystemAggregateFunctionCombinators
is_temporary: 0
data_paths: []
metadata_path: /var/lib/clickhouse/metadata/system/aggregate_function_combinators.sql
metadata_modification_time: 1970-01-01 03:00:00
dependencies_database: []
dependencies_table: []
create_table_query:
engine_full:
partition_key:
sorting_key:
primary_key:
sampling_key:
storage_policy:
total_rows: ᴺᵁᴸᴸ
total_bytes: ᴺᵁᴸᴸ
Row 2:
──────
database: system
name: asynchronous_metrics
uuid: 00000000-0000-0000-0000-000000000000
engine: SystemAsynchronousMetrics
is_temporary: 0
data_paths: []
metadata_path: /var/lib/clickhouse/metadata/system/asynchronous_metrics.sql
metadata_modification_time: 1970-01-01 03:00:00
dependencies_database: []
dependencies_table: []
create_table_query:
engine_full:
partition_key:
sorting_key:
primary_key:
sampling_key:
storage_policy:
total_rows: ᴺᵁᴸᴸ
total_bytes: ᴺᵁᴸᴸ
2 rows in set. Elapsed: 0.004 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/tables) <!--hide-->

View File

@ -9,6 +9,12 @@ Connects to a ClickHouse server and repeatedly sends specified queries.
Syntax:
``` bash
$ clickhouse-benchmark --query ["single query"] [keys]
```
or
``` bash
$ echo "single query" | clickhouse-benchmark [keys]
```
@ -34,6 +40,7 @@ clickhouse-benchmark [keys] < queries_file
## Keys {#clickhouse-benchmark-keys}
- `--query=WORD` - Query to execute. If this parameter is not passed clickhouse-benchmark will read queries from standard input.
- `-c N`, `--concurrency=N` — Number of queries that `clickhouse-benchmark` sends simultaneously. Default value: 1.
- `-d N`, `--delay=N` — Interval in seconds between intermediate reports (set 0 to disable reports). Default value: 1.
- `-h WORD`, `--host=WORD` — Server host. Default value: `localhost`. For the [comparison mode](#clickhouse-benchmark-comparison-mode) you can use multiple `-h` keys.

View File

@ -591,3 +591,7 @@ Result:
```
[Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) <!--hide-->
## countMatches(haystack, pattern) {#countmatcheshaystack-pattern}
Returns the number of regular expression matches for a `pattern` in a `haystack`.

View File

@ -9,7 +9,7 @@ toc_title: DELETE
ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr
```
Allows to delete data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Deletes data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
!!! note "Note"
The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use.

View File

@ -19,5 +19,4 @@ The first two commands are lightweight in a sense that they only change metadata
Also, they are replicated, syncing indices metadata via ZooKeeper.
!!! note "Note"
Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including
[replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants).
Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants).

View File

@ -241,6 +241,46 @@ ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
## UPDATE IN PARTITION {#update-in-partition}
Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Syntax:
``` sql
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr
```
### Example
``` sql
ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2;
```
### See Also
- [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements)
## DELETE IN PARTITION {#delete-in-partition}
Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Syntax:
``` sql
ALTER TABLE [db.]table DELETE [IN PARTITION partition_id] WHERE filter_expr
```
### Example
``` sql
ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2;
```
### See Also
- [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations)
## How to Set Partition Expression {#alter-how-to-specify-part-expr}
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:
@ -258,4 +298,6 @@ All the rules above are also true for the [OPTIMIZE](../../../sql-reference/stat
OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL;
```
`IN PARTITION` specifies the partition to which the [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) or [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) expressions are applied as a result of the `ALTER TABLE` query. New parts are created only from the specified partition. In this way, `IN PARTITION` helps to reduce the load when the table is divided into many partitions, and you only need to update the data point-by-point.
The examples of `ALTER ... PARTITION` queries are demonstrated in the tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql).

View File

@ -9,7 +9,7 @@ toc_title: UPDATE
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr
```
Allows to manipulate data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Manipulates data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
!!! note "Note"
The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use.

View File

@ -11,7 +11,7 @@ Syntax:
``` sql
CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[KEYED BY {'none' | 'user name' | 'ip address' | 'forwarded ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]

View File

@ -0,0 +1,150 @@
---
toc_priority: 39
toc_title: EXPLAIN
---
# EXPLAIN Statement {#explain}
Show the execution plan of a statement.
Syntax:
```sql
EXPLAIN [AST | SYNTAX | PLAN | PIPELINE] [setting = value, ...] SELECT ... [FORMAT ...]
```
Example:
```sql
EXPLAIN SELECT sum(number) FROM numbers(10) UNION ALL SELECT sum(number) FROM numbers(10) ORDER BY sum(number) ASC FORMAT TSV;
```
```sql
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemNumbers)
Expression (Projection)
MergingSorted (Merge sorted streams for ORDER BY)
MergeSorting (Merge sorted blocks for ORDER BY)
PartialSorting (Sort each block for ORDER BY)
Expression (Before ORDER BY and SELECT)
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemNumbers)
```
## EXPLAIN Types {#explain-types}
- `AST` — Abstract syntax tree.
- `SYNTAX` — Query text after AST-level optimizations.
- `PLAN` — Query execution plan.
- `PIPELINE` — Query execution pipeline.
### EXPLAIN AST {#explain-ast}
Dump query AST.
Example:
```sql
EXPLAIN AST SELECT 1;
```
```sql
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 1)
ExpressionList (children 1)
Literal UInt64_1
```
### EXPLAIN SYNTAX {#explain-syntax}
Return query after syntax optimizations.
Example:
```sql
EXPLAIN SYNTAX SELECT * FROM system.numbers AS a, system.numbers AS b, system.numbers AS c;
```
```sql
SELECT
`--a.number` AS `a.number`,
`--b.number` AS `b.number`,
number AS `c.number`
FROM
(
SELECT
number AS `--a.number`,
b.number AS `--b.number`
FROM system.numbers AS a
CROSS JOIN system.numbers AS b
) AS `--.s`
CROSS JOIN system.numbers AS c
```
### EXPLAIN PLAN {#explain-plan}
Dump query plan steps.
Settings:
- `header` — Print output header for step. Default: 0.
- `description` — Print step description. Default: 1.
- `actions` — Print detailed information about step actions. Default: 0.
Example:
```sql
EXPLAIN SELECT sum(number) FROM numbers(10) GROUP BY number % 4;
```
```sql
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemNumbers)
```
!!! note "Note"
Step and query cost estimation is not supported.
### EXPLAIN PIPELINE {#explain-pipeline}
Settings:
- `header` — Print header for each output port. Default: 0.
- `graph` — Use DOT graph description language. Default: 0.
- `compact` — Print graph in compact mode if graph is enabled. Default: 1.
Example:
```sql
EXPLAIN PIPELINE SELECT sum(number) FROM numbers_mt(100000) GROUP BY number % 4;
```
```sql
(Union)
(Expression)
ExpressionTransform
(Expression)
ExpressionTransform
(Aggregating)
Resize 2 → 1
AggregatingTransform × 2
(Expression)
ExpressionTransform × 2
(SettingQuotaAndLimits)
(ReadFromStorage)
NumbersMt × 2 0 → 1
```
[Оriginal article](https://clickhouse.tech/docs/en/sql-reference/statements/explain/) <!--hide-->

View File

@ -29,3 +29,4 @@ Statements represent various kinds of action you can perform using SQL queries.
- [SET ROLE](../../sql-reference/statements/set-role.md)
- [TRUNCATE](../../sql-reference/statements/truncate.md)
- [USE](../../sql-reference/statements/use.md)
- [EXPLAIN](../../sql-reference/statements/explain.md)

View File

@ -152,7 +152,7 @@ ClickHouse can manage background processes in [MergeTree](../../engines/table-en
Provides possibility to stop background merges for tables in the MergeTree family:
``` sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
!!! note "Note"
@ -163,7 +163,7 @@ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
Provides possibility to start background merges for tables in the MergeTree family:
``` sql
SYSTEM START MERGES [[db.]merge_tree_family_table_name]
SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
### STOP TTL MERGES {#query_language-stop-ttl-merges}

View File

@ -57,7 +57,7 @@ Identifiers are:
Identifiers can be quoted or non-quoted. The latter is preferred.
Non-quoted identifiers must match the regex `^[0-9a-zA-Z_]*[a-zA-Z_]$` and can not be equal to [keywords](#syntax-keywords). Examples: `x, _1, X_y__Z123_.`
Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x`, `_1`, `X_y__Z123_`.
If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``.

View File

@ -177,6 +177,10 @@ Marks numbers: 0 1 2 3 4 5 6 7 8
ClickHouse не требует уникального первичного ключа. Можно вставить много строк с одинаковым первичным ключом.
Ключ в `PRIMARY KEY` и `ORDER BY` может иметь тип `Nullable`. За поддержку этой возможности отвечает настройка [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key).
При сортировке с использованием выражения `ORDER BY` для значений `NULL` всегда работает принцип [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values).
### Выбор первичного ключа {#vybor-pervichnogo-kliucha}
Количество столбцов в первичном ключе не ограничено явным образом. В зависимости от структуры данных в первичный ключ можно включать больше или меньше столбцов. Это может:
@ -565,6 +569,7 @@ ALTER TABLE example_table
- `disk` — диск, находящийся внутри тома.
- `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома.
- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1).
- `prefer_not_to_merge` — Отключает слияние кусков данных, хранящихся на данном томе. Если данная настройка включена, то слияние данных, хранящихся на данном томе, не допускается. Это позволяет контролировать работу ClickHouse с медленными дисками.
Примеры конфигураций:
@ -593,6 +598,19 @@ ALTER TABLE example_table
</volumes>
<move_factor>0.2</move_factor>
</moving_from_ssd_to_hdd>
<small_jbod_with_external_no_merges>
<volumes>
<main>
<disk>jbod1</disk>
</main>
<external>
<disk>external</disk>
<prefer_not_to_merge>true</prefer_not_to_merge>
</external>
</volumes>
</small_jbod_with_external_no_merges>
</policies>
...
</storage_configuration>

View File

@ -2235,4 +2235,15 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x);
Значение по умолчанию: `\N`.
## allow_nullable_key {#allow-nullable-key}
Включает или отключает поддержку типа [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable) для ключей таблиц [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree).
Возможные значения:
- 1 — включает поддержку типа `Nullable` для ключей таблиц.
- 0 — отключает поддержку типа `Nullable` для ключей таблиц.
Значение по умолчанию: `0`.
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->

View File

@ -10,6 +10,7 @@
- `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — имена дисков, содержащихся в политике хранения.
- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — максимальный размер куска данных, который может храниться на дисках тома (0 — без ограничений).
- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1).
- `prefer_not_to_merge` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Значение настройки `prefer_not_to_merge`. Если данная настройка включена, то слияние данных, хранящихся на данном томе, не допускается. Это позволяет контролировать работу ClickHouse с медленными дисками.
Если политика хранения содержит несколько томов, то каждому тому соответствует отдельная запись в таблице.

View File

@ -9,7 +9,7 @@ toc_title: DELETE
ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr
```
Позволяет удалить данные, соответствующие указанному выражению фильтрации. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations).
Удаляет данные, соответствующие указанному выражению фильтрации. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations).
!!! note "Note"
Префикс `ALTER TABLE` делает этот синтаксис отличным от большинства других систем, поддерживающих SQL. Он предназначен для обозначения того, что в отличие от аналогичных запросов в базах данных OLTP это тяжелая операция, не предназначенная для частого использования.

View File

@ -243,6 +243,46 @@ ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
## UPDATE IN PARTITION {#update-in-partition}
Манипулирует данными в указанной партиции, соответствующими заданному выражению фильтрации. Реализовано как мутация [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Синтаксис:
``` sql
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr
```
### Пример
``` sql
ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2;
```
### Смотрите также
- [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements)
## DELETE IN PARTITION {#delete-in-partition}
Удаляет данные в указанной партиции, соответствующие указанному выражению фильтрации. Реализовано как мутация [mutation](../../../sql-reference/statements/alter/index.md#mutations).
Синтаксис:
``` sql
ALTER TABLE [db.]table DELETE [IN PARTITION partition_id] WHERE filter_expr
```
### Пример
``` sql
ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2;
```
### Смотрите также
- [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations)
## Как задавать имя партиции в запросах ALTER {#alter-how-to-specify-part-expr}
Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать:
@ -262,6 +302,8 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL;
```
`IN PARTITION` указывает на партицию, для которой применяются выражения [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) или [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) в результате запроса `ALTER TABLE`. Новые куски создаются только в указанной партиции. Таким образом, `IN PARTITION` помогает снизить нагрузку, когда таблица разбита на множество партиций, а вам нужно обновить данные лишь точечно.
Примеры запросов `ALTER ... PARTITION` можно посмотреть в тестах: [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) и [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql).
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/alter/partition/) <!--hide-->

View File

@ -9,7 +9,7 @@ toc_title: UPDATE
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr
```
Позволяет манипулировать данными, соответствующими заданному выражению фильтрации. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations).
Манипулирует данными, соответствующими заданному выражению фильтрации. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations).
!!! note "Note"
Префикс `ALTER TABLE` делает этот синтаксис отличным от большинства других систем, поддерживающих SQL. Он предназначен для обозначения того, что в отличие от аналогичных запросов в базах данных OLTP это тяжелая операция, не предназначенная для частого использования.

View File

@ -130,7 +130,7 @@ ClickHouse может управлять фоновыми процессами
Позволяет остановить фоновые мержи для таблиц семейства MergeTree:
``` sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
!!! note "Note"
@ -141,7 +141,7 @@ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
Включает фоновые мержи для таблиц семейства MergeTree:
``` sql
SYSTEM START MERGES [[db.]merge_tree_family_table_name]
SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
### STOP TTL MERGES {#query_language-stop-ttl-merges}

View File

@ -1,12 +1,36 @@
---
toc_priority: 14
toc_title: "\u266A\u64CD\u573A\u266A"
toc_title: 体验平台
---
# ClickHouse体验平台 {#clickhouse-playground}
[ClickHouse体验平台](https://play.clickhouse.tech?file=welcome) 允许人们通过即时运行查询来尝试ClickHouse而无需设置他们的服务器或集群。
体验平台中提供了几个示例数据集以及显示ClickHouse特性的示例查询。
体验平台中提供几个示例数据集以及显示ClickHouse特性的示例查询。还有一些ClickHouse LTS版本可供尝试。
ClickHouse体验平台提供了小型集群[Managed Service for ClickHouse](https://cloud.yandex.com/services/managed-clickhouse)实例配置(4 vCPU, 32 GB RAM)它们托管在[Yandex.Cloud](https://cloud.yandex.com/). 更多信息查询[cloud providers](../commercial/cloud.md).
您可以使用任何HTTP客户端对ClickHouse体验平台进行查询例如[curl](https://curl.haxx.se)或者[wget](https://www.gnu.org/software/wget/),或使用[JDBC](../interfaces/jdbc.md)或者[ODBC](../interfaces/odbc.md)驱动连接。关于支持ClickHouse的软件产品的更多信息详见[here](../interfaces/index.md).
## Credentials {#credentials}
| 参数 | 值 |
|:--------------------|:----------------------------------------|
| HTTPS端点 | `https://play-api.clickhouse.tech:8443` |
| TCP端点 | `play-api.clickhouse.tech:9440` |
| 用户 | `playground` |
| 密码 | `clickhouse` |
还有一些带有特定ClickHouse版本的附加信息来试验它们之间的差异(端口和用户/密码与上面相同):
- 20.3 LTS: `play-api-v20-3.clickhouse.tech`
- 19.14 LTS: `play-api-v19-14.clickhouse.tech`
!!! note "注意"
所有这些端点都需要安全的TLS连接。
## 查询限制 {#limitations}
查询以只读用户身份执行。 这意味着一些局限性:
@ -14,33 +38,34 @@ toc_title: "\u266A\u64CD\u573A\u266A"
- 不允许插入查询
还强制执行以下设置:
- [`max_result_bytes=10485760`](../operations/settings/query_complexity/#max-result-bytes)
- [`max_result_rows=2000`](../operations/settings/query_complexity/#setting-max_result_rows)
- [`result_overflow_mode=break`](../operations/settings/query_complexity/#result-overflow-mode)
- [`max_execution_time=60000`](../operations/settings/query_complexity/#max-execution-time)
- [max_result_bytes=10485760](../operations/settings/query-complexity/#max-result-bytes)
- [max_result_rows=2000](../operations/settings/query-complexity/#setting-max_result_rows)
- [result_overflow_mode=break](../operations/settings/query-complexity/#result-overflow-mode)
- [max_execution_time=60000](../operations/settings/query-complexity/#max-execution-time)
ClickHouse体验还有如下
[ClickHouse管理服务](https://cloud.yandex.com/services/managed-clickhouse)
实例托管 [Yandex云](https://cloud.yandex.com/)。
更多信息 [云提供商](../commercial/cloud.md)。
ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的.
体验平台后端只是一个ClickHouse集群没有任何额外的服务器端应用程序。
体验平台也同样提供了ClickHouse HTTPS服务端口。
## 示例 {#examples}
您可以使用任何HTTP客户端向体验平台进行查询例如 [curl](https://curl.haxx.se) 或 [wget](https://www.gnu.org/software/wget/),或使用以下方式建立连接 [JDBC](../interfaces/jdbc.md) 或 [ODBC](../interfaces/odbc.md) 驱动。
有关支持ClickHouse的软件产品的更多信息请访问 [这里](../interfaces/index.md)。
| 参数 | 值 |
|:---------|:--------------------------------------|
| 服务端口 | https://play-api.clickhouse.tech:8443 |
| 用户 | `playground` |
| 密码 | `clickhouse` |
请注意,此服务端口需要安全连接。
示例:
使用`curl`连接Https服务
``` bash
curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse!';&user=playground&password=clickhouse&database=datasets"
curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse\!';&user=playground&password=clickhouse&database=datasets"
```
TCP连接示例[CLI](../interfaces/cli.md):
``` bash
clickhouse client --secure -h play-api.clickhouse.tech --port 9440 -u playground --password clickhouse -q "SELECT 'Play ClickHouse\!'"
```
## Implementation Details {#implementation-details}
ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的。
ClickHouse体验平台是一个ClickHouse集群没有任何附加的服务器端应用程序。如上所述ClickHouse的HTTPS和TCP/TLS端点也可以作为体验平台的一部分公开使用, 代理通过[Cloudflare Spectrum](https://www.cloudflare.com/products/cloudflare-spectrum/)增加一层额外的保护和改善连接。
!!! warning "注意"
**强烈不推荐**在任何其他情况下将ClickHouse服务器暴露给公共互联网。确保它只在私有网络上侦听并由正确配置的防火墙监控。

View File

@ -1,71 +1,105 @@
# 命令行客户端 {#ming-ling-xing-ke-hu-duan}
---
toc_priority: 17
toc_title: 命令行客户端
---
通过命令行来访问 ClickHouse您可以使用 `clickhouse-client`
# 命令行客户端 {#command-line-client}
ClickHouse提供了一个原生命令行客户端`clickhouse-client`客户端支持命令行支持的更多信息详见[Configuring](#interfaces_cli_configuration)。
[安装部署](../getting-started/index.md)后,系统默认会安装`clickhouse-client`(同时它属于`clickhouse-client`安装包中)。
``` bash
$ clickhouse-client
ClickHouse client version 0.0.26176.
Connecting to localhost:9000.
Connected to ClickHouse server version 0.0.26176.:)
ClickHouse client version 19.17.1.1579 (official build).
Connecting to localhost:9000 as user default.
Connected to ClickHouse server version 19.17.1 revision 54428.
:)
```
该客户端支持命令行参数以及配置文件。查看更多,请看 «[配置](#interfaces_cli_configuration)»
不同的客户端和服务器版本彼此兼容,但是一些特性可能在旧客户机中不可用。我们建议使用与服务器应用相同版本的客户端。当你尝试使用旧版本的客户端时,服务器上的`clickhouse-client`会显示如下信息:
## 使用方式 {#shi-yong-fang-shi}
ClickHouse client version is older than ClickHouse server. It may lack support for new features.
这个客户端可以选择使用交互式与非交互式(批量)两种模式。
使用批量模式,要指定 `query` 参数,或者发送数据到 `stdin`(它会检查 `stdin` 是否是 Terminal或者两种同时使用。
它与 HTTP 接口很相似,当使用 `query` 参数发送数据到 `stdin` 时,客户端请求就是一行一行的 `stdin` 输入作为 `query` 的参数。这种方式在大规模的插入请求中非常方便。
## 使用方式 {#cli_usage}
使用这个客户端插入数据的示例:
客户端可以在交互和非交互(批处理)模式下使用。要使用批处理模式,请指定`query`参数,或将数据发送到`stdin`(它会验证`stdin`是否是终端)或两者同时进行。与HTTP接口类似当使用`query`参数并向`stdin`发送数据时,客户端请求就是一行一行的`stdin`输入作为`query`的参数。这种方式在大规模的插入请求中非常方便。
使用客户端插入数据的示例:
``` bash
echo -ne "1, 'some text', '2016-08-14 00:00:00'\n2, 'some more text', '2016-08-14 00:00:01'" | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
$ echo -ne "1, 'some text', '2016-08-14 00:00:00'\n2, 'some more text', '2016-08-14 00:00:01'" | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
cat <<_EOF | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
$ cat <<_EOF | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
3, 'some text', '2016-08-14 00:00:00'
4, 'some more text', '2016-08-14 00:00:01'
_EOF
cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
$ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV";
```
在批量模式中,默认的数据格式是 `TabSeparated` 分隔的。您可以根据查询来灵活设置 FORMAT 格式。
在批量模式中,默认的数据格式是`TabSeparated`分隔的。您可以根据查询来灵活设置FORMAT格式。
默认情况下,在批量模式中只能执行单个查询。为了从一个 Script 中执行多个查询,可以使用 `--multiquery` 参数。除了 INSERT 请求外,这种方式在任何地方都有用。查询的结果会连续且不含分隔符地输出。
同样的,为了执行大规模的查询,您可以为每个查询执行一次 `clickhouse-client`。但注意到每次启动 `clickhouse-client` 程序都需要消耗几十毫秒时间。
默认情况下在批量模式中只能执行单个查询。为了从一个Script中执行多个查询可以使用`--multiquery`参数。除了INSERT请求外这种方式在任何地方都有用。查询的结果会连续且不含分隔符地输出。
同样的,为了执行大规模的查询,您可以为每个查询执行一次`clickhouse-client`。但注意到每次启动`clickhouse-client`程序都需要消耗几十毫秒时间。
在交互模式下,每条查询过后,你可以直接输入下一条查询命令。
如果 `multiline` 没有指定(默认没指定):为了执行查询,按下 Enter 即可。查询语句不是必须使用分号结尾。如果需要写一个多行的查询语句,可以在换行之前输入一个反斜杠`\`,然后在您按下 Enter 键后,您就可以输入当前语句的下一行查询了。
如果`multiline`没有指定默认没指定为了执行查询按下Enter即可。查询语句不是必须使用分号结尾。如果需要写一个多行的查询语句可以在换行之前输入一个反斜杠`\`然后在您按下Enter键后您就可以输入当前语句的下一行查询了。
如果 `multiline` 指定了:为了执行查询,需要以分号结尾并且按下 Enter 键。如果行末没有分号,将认为当前语句并没有输入完而要求继续输入下一行。
如果指定了`multiline`为了执行查询需要以分号结尾并且按下Enter键。如果行末没有分号将认为当前语句并没有输入完而要求继续输入下一行。
若只运行单个查询,分号后面的所有内容都会被忽略。
您可以指定 `\G` 来替代分号或者在分号后面,这表示 `Vertical` 的格式。在这种格式下,每一个值都会打印在不同的行中,这种方式对于宽表来说很方便。这个不常见的特性是为了兼容 MySQL 命令而加的。
您可以指定`\G`来替代分号或者在分号后面,这表示使用`Vertical`的格式。在这种格式下每一个值都会打印在不同的行中这种方式对于宽表来说很方便。这个不常见的特性是为了兼容MySQL命令而加的。
命令行客户端是基于 `replxx`。换句话说,它可以使用我们熟悉的快捷键方式来操作以及保留历史命令。
历史命令会写入在 `~/.clickhouse-client-history` 中。
命令行客户端是基于`replxx`(类似于`readline`)。换句话说,它可以使用我们熟悉的快捷键方式来操作以及保留历史命令。
历史命令会写入在`~/.clickhouse-client-history`中。
默认情况下,输出的格式是 `PrettyCompact`。您可以通过 FORMAT 设置根据不同查询来修改格式,或者通过在查询末尾指定 `\G` 字符,或通过在命令行中使用 `--format` `--vertical` 参数,或使用客户端的配置文件。
默认情况下,输出的格式是`PrettyCompact`。您可以通过FORMAT设置根据不同查询来修改格式或者通过在查询末尾指定`\G`字符,或通过在命令行中使用`--format`或`--vertical`参数,或使用客户端的配置文件。
若要退出客户端,使用 Ctrl+D (或 Ctrl+C或者输入以下其中一个命令`exit`, `quit`, `logout`, `учше`, `йгше`, `дщпщге`, `exit;`, `quit;`, `logout;`, `учшеж`, `йгшеж`, `дщпщгеж`, `q`, `й`, `q`, `Q`, `:q`, `й`, `Й`, `Жй`
若要退出客户端使用Ctrl+D或Ctrl+C或者输入以下其中一个命令`exit`, `quit`, `logout`, `учше`, `йгше`, `дщпщге`, `exit;`, `quit;`, `logout;`, `q`, `Q`, `:q`
当执行一个查询的时候,客户端会显示:
1. 进度, 进度会每秒更新十次 (默认情况下)。 对于很快的查询,进度可能没有时间显示。
1. 进度, 进度会每秒更新十次(默认情况下)。对于很快的查询,进度可能没有时间显示。
2. 为了调试会显示解析且格式化后的查询语句。
3. 指定格式的输出结果。
4. 输出结果的行数的行数,经过的时间,以及查询处理的速度。
您可以通过 Ctrl+C 来取消一个长时间的查询。然而,您依然需要等待服务端来中止请求。在某个阶段去取消查询是不可能的。如果您不等待并再次按下 Ctrl + C客户端将会退出。
您可以通过Ctrl+C来取消一个长时间的查询。然而您依然需要等待服务端来中止请求。在某个阶段去取消查询是不可能的。如果您不等待并再次按下Ctrl + C,客户端将会退出。
命令行客户端允许通过外部数据 (外部临时表) 来查询。更多相关信息,请参考 «[外部数据查询处理](../engines/table-engines/special/external-data.md)».
命令行客户端允许通过外部数据(外部临时表)来查询。更多相关信息,请参考 «[外部数据查询处理](../engines/table-engines/special/external-data.md)».
### 查询参数 {#cli-queries-with-parameters}
您可以创建带有参数的查询,并将值从客户端传递给服务器。这允许避免在客户端使用特定的动态值格式化查询。例如:
``` bash
$ clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}"
```
#### 查询语法 {#cli-queries-with-parameters-syntax}
像平常一样格式化一个查询然后把你想要从app参数传递到查询的值用大括号格式化格式如下:
``` sql
{<name>:<data type>}
```
- `name` — 占位符标识符。在控制台客户端,使用`--param_<name> = value`来指定
- `data type` — [数据类型](../sql-reference/data-types/index.md)参数值。例如,一个数据结构`(integer, ('string', integer))`拥有`Tuple(UInt8, Tuple(String, UInt8))`数据类型(你也可以用另一个[integer](../sql-reference/data-types/int-uint.md)类型)。
#### 示例 {#example}
``` bash
$ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}"
```
## 配置 {#interfaces_cli_configuration}
您可以通过以下方式传入参数到 `clickhouse-client` 中 (所有的参数都有默认值):
您可以通过以下方式传入参数到`clickhouse-client`中(所有的参数都有默认值):
- 通过命令行
@ -75,29 +109,32 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
配置文件的配置会覆盖默认值
### 命令行参数 {#ming-ling-xing-can-shu}
### 命令行参数 {#command-line-options}
- `--host, -h` - 服务端的 host 名称, 默认是 localhost。 您可以选择使用 host 名称或者 IPv4 或 IPv6 地址。
- `--port` 连接的端口,默认值: 9000。注意 HTTP 接口以及 TCP 原生接口使用不同端口
- `--user, -u` 用户名。 默认值: default
- `--password` 密码。 默认值: 空字符串。
- `--query, -q` 非交互模式下的查询语句.
- `--database, -d` 默认当前操作的数据库. 默认值: 服务端默认的配置 (默认是 `default`)。
- `--multiline, -m` 如果指定允许多行语句查询Enter 仅代表换行,不代表查询语句完结)。
- `--multiquery, -n` 如果指定, 允许处理用号分隔的多个查询,只在非交互模式下生效。
- `--host, -h` - 服务端的host名称, 默认是`localhost`。您可以选择使用host名称或者IPv4或IPv6地址。
- `--port` 连接的端口默认值9000。注意HTTP接口以及TCP原生接口使用的是不同端口。
- `--user, -u` 用户名。 默认值:`default`
- `--password` 密码。 默认值:空字符串。
- `--query, -q` 使用非交互模式查询。
- `--database, -d` 默认当前操作的数据库. 默认值:服务端默认的配置(默认是`default`)。
- `--multiline, -m` 如果指定允许多行语句查询Enter仅代表换行不代表查询语句完结
- `--multiquery, -n` 如果指定, 允许处理用`;`号分隔的多个查询,只在非交互模式下生效。
- `--format, -f` 使用指定的默认格式输出结果。
- `--vertical, -E` 如果指定,默认情况下使用垂直格式输出结果。这与 format=Vertical 相同。在这种格式中,每个值都在单独的行上打印,这种方式对显示宽表很有帮助。
- `--time, -t` 如果指定,非交互模式下会打印查询执行的时间到 stderr 中。
- `--vertical, -E` 如果指定,默认情况下使用垂直格式输出结果。这与`format=Vertical`相同。在这种格式中,每个值都在单独的行上打印,这种方式对显示宽表很有帮助。
- `--time, -t` 如果指定,非交互模式下会打印查询执行的时间到`stderr`中。
- `--stacktrace` 如果指定,如果出现异常,会打印堆栈跟踪信息。
- `--config-file` 配置文件的名称。
- `--secure` 如果指定,将通过安全连接连接到服务器。
- `--history_file` — 存放命令历史的文件的路径。
- `--param_<name>` — 查询参数配置[查询参数](#cli-queries-with-parameters).
### 配置文件 {#pei-zhi-wen-jian}
### 配置文件 {#configuration_files}
`clickhouse-client` 使用一下第一个存在的文件:
`clickhouse-client`使用以下第一个配置文件:
- 通过 `--config-file` 参数指定的文件.
- 通过`--config-file`参数指定。
- `./clickhouse-client.xml`
- `\~/.clickhouse-client/config.xml`
- `~/.clickhouse-client/config.xml`
- `/etc/clickhouse-client/config.xml`
配置文件示例:
@ -106,6 +143,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
<config>
<user>username</user>
<password>password</password>
<secure>False</secure>
</config>
```

View File

@ -1,5 +1,10 @@
# C ++客户端库 {#c-ke-hu-duan-ku}
---
toc_priority: 24
toc_title: C++客户端库
---
请参阅以下网站的自述文件[ツ暗ェツ氾环催ツ団](https://github.com/ClickHouse/clickhouse-cpp)资料库。
# C++客户端库 {#c-client-library}
请参考仓库的描述文件[clickhouse-cpp](https://github.com/ClickHouse/clickhouse-cpp)。
[原始文章](https://clickhouse.tech/docs/zh/interfaces/cpp/) <!--hide-->

View File

@ -1,23 +1,31 @@
# HTTP 客户端 {#http-ke-hu-duan}
---
toc_priority: 19
toc_title: HTTP客户端
---
HTTP 接口可以让你通过任何平台和编程语言来使用 ClickHouse。我们用 Java 和 Perl 以及 shell 脚本来访问它。在其他的部门中HTTP 接口会用在 PerlPython 以及 Go 中。HTTP 接口比 TCP 原生接口更为局限,但是却有更好的兼容性。
# HTTP客户端 {#http-interface}
默认情况下clickhouse-server 会在端口 8123 上监控 HTTP 请求(这可以在配置中修改)。
如果你发送了一个不带参数的 GET 请求,它会返回一个字符串 «Ok.»(结尾有换行)。可以将它用在健康检查脚本中。
HTTP接口允许您在任何编程语言的任何平台上使用ClickHouse。我们使用它在Java和Perl以及shell脚本中工作。在其他部门中HTTP接口用于Perl、Python和Go。HTTP接口比原生接口受到更多的限制但它具有更好的兼容性。
默认情况下,`clickhouse-server`会在`8123`端口上监控HTTP请求这可以在配置中修改
如果你发送了一个未携带任何参数的`GET /`请求,它会返回一个字符串 «Ok.»(结尾有换行)。可以将它用在健康检查脚本中。
如果你发送了一个未携带任何参数的`GET /`请求它返回响应码200和`OK`字符串定义,可在[Http服务响应配置](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response)定义(在末尾添加换行)
``` bash
$ curl 'http://localhost:8123/'
Ok.
```
通过 URL 中的 `query` 参数来发送请求,或者发送 POST 请求,或者将查询的开头部分放在 URL `query` 参数中,其他部分放在 POST 我们会在后面解释为什么这样做是有必要的。URL 的大小会限制在 16 KB所以发送大型查询时要时刻记住这点。
通过URL中的 `query` 参数来发送请求或者发送POST请求或者将查询的开头部分放在URL的`query`参数中其他部分放在POST中我们会在后面解释为什么这样做是有必要的。URL的大小会限制在16KB所以发送大型查询时要时刻记住这点。
如果请求成功,将会收到 200 的响应状态码和响应主体中的结果。
如果发生了某个异常,将会收到 500 的响应状态码和响应主体中的异常描述信息。
如果请求成功将会收到200的响应状态码和响应主体中的结果。
如果发生了某个异常将会收到500的响应状态码和响应主体中的异常描述信息。
当使用 GET 方法请求时,`readonly` 会被设置。换句话说,若要作修改数据的查询,只能发送 POST 方法的请求。可以将查询通过 POST 主体发送,也可以通过 URL 参数发送。
当使用GET方法请求时`readonly`会被设置。换句话说若要作修改数据的查询只能发送POST方法的请求。可以将查询通过POST主体发送也可以通过URL参数发送。
例:
例:
``` bash
$ curl 'http://localhost:8123/?query=SELECT%201'
@ -26,13 +34,14 @@ $ curl 'http://localhost:8123/?query=SELECT%201'
$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1'
1
$ GET 'http://localhost:8123/?query=SELECT 1'
1
$ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123
HTTP/1.0 200 OK
Date: Wed, 27 Nov 2019 10:30:18 GMT
Connection: Close
Date: Fri, 16 Nov 2012 19:21:50 GMT
Content-Type: text/tab-separated-values; charset=UTF-8
X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal
X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
1
```
@ -50,7 +59,22 @@ $ echo '1' | curl 'http://localhost:8123/?query=SELECT' --data-binary @-
1
```
如果一部分请求是通过参数发送的,另外一部分通过 POST 主体发送,两部分查询之间会一行空行插入。
如您所见curl有些不方便因为空格必须进行URL转义。
尽管wget本身会对所有内容进行转义但我们不推荐使用它因为在使用keepalive和传输编码chunked时它在HTTP 1.1上不能很好地工作。
``` bash
$ echo 'SELECT 1' | curl 'http://localhost:8123/' --data-binary @-
1
$ echo 'SELECT 1' | curl 'http://localhost:8123/?query=' --data-binary @-
1
$ echo '1' | curl 'http://localhost:8123/?query=SELECT' --data-binary @-
1
```
如果部分查询是在参数中发送的部分是在POST中发送的则在这两个数据部分之间插入换行。
错误示例:
``` bash
@ -60,8 +84,11 @@ ECT 1
, expected One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE., e.what() = DB::Exception
```
默认情况下,返回的数据是 TabSeparated 格式的,更多信息,见 «\[数据格式\]» 部分。
可以使用 FORMAT 设置查询来请求不同格式。
默认情况下,返回的数据是`TabSeparated`格式的,更多信息,见[Formats](../interfaces/formats/)部分。
您可以使用查询的FORMAT子句来设置其他格式。
另外,还可以使用`default_format`URL参数或`X-ClickHouse-Format`头来指定TabSeparated之外的默认格式。
``` bash
$ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @-
@ -72,42 +99,42 @@ $ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @
└───┘
```
INSERT 必须通过 POST 方法来插入数据。这种情况下,你可以将查询的开头部分放在 URL 参数中,然后用 POST 主体传入插入的数据。插入的数据可以是,举个例子,从 MySQL 导出的以 tab 分割的数据。在这种方式中INSERT 查询取代了 LOAD DATA LOCAL INFILE from MySQL
INSERT必须通过POST方法来插入数据。在这种情况下您可以在URL参数中编写查询的开始部分并使用POST传递要插入的数据。例如要插入的数据可以是来自MySQL的一个以tab分隔的存储。通过这种方式INSERT查询替换了从MySQL查询的LOAD DATA LOCAL INFILE
示例: 创建一个表:
``` bash
echo 'CREATE TABLE t (a UInt8) ENGINE = Memory' | POST 'http://localhost:8123/'
$ echo 'CREATE TABLE t (a UInt8) ENGINE = Memory' | curl 'http://localhost:8123/' --data-binary @-
```
使用类似 INSERT 的查询来插入数据:
使用类似INSERT的查询来插入数据
``` bash
echo 'INSERT INTO t VALUES (1),(2),(3)' | POST 'http://localhost:8123/'
$ echo 'INSERT INTO t VALUES (1),(2),(3)' | curl 'http://localhost:8123/' --data-binary @-
```
数据可以从查询中单独发送:
``` bash
echo '(4),(5),(6)' | POST 'http://localhost:8123/?query=INSERT INTO t VALUES'
$ echo '(4),(5),(6)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20VALUES' --data-binary @-
```
可以指定任何数据格式。值的格式和写入表 `t` 的值的格式相同:
您可以指定任何数据格式。`Values`格式与将INSERT写入`t`值时使用的格式相同:
``` bash
echo '(7),(8),(9)' | POST 'http://localhost:8123/?query=INSERT INTO t FORMAT Values'
$ echo '(7),(8),(9)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20FORMAT%20Values' --data-binary @-
```
若要插入 tab 分割的数据,需要指定对应的格式:
若要插入tab分割的数据需要指定对应的格式
``` bash
echo -ne '10\n11\n12\n' | POST 'http://localhost:8123/?query=INSERT INTO t FORMAT TabSeparated'
$ echo -ne '10\n11\n12\n' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20FORMAT%20TabSeparated' --data-binary @-
```
从表中读取内容。由于查询处理是并行的,数据以随机顺序输出。
``` bash
$ GET 'http://localhost:8123/?query=SELECT a FROM t'
$ curl 'http://localhost:8123/?query=SELECT%20a%20FROM%20t'
7
8
9
@ -122,22 +149,37 @@ $ GET 'http://localhost:8123/?query=SELECT a FROM t'
6
```
删除表
删除表
``` bash
POST 'http://localhost:8123/?query=DROP TABLE t'
$ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @-
```
成功请求后并不会返回数据,返回一个空的响应体。
可以通过压缩来传输数据。压缩的数据没有一个标准的格式,但你需要指定一个压缩程序来使用它(sudo apt-get install compressor-metrika-yandex
在传输数据时可以使用ClickHouse内部压缩格式。压缩的数据具有非标准格式您需要使用特殊的`clickhouse-compressor`程序来处理它(它是与`clickhouse-client`包一起安装的)。为了提高数据插入的效率,您可以通过使用[http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress)设置禁用服务器端校验
如果在 URL 中指定了 `compress=1` ,服务会返回压缩的数据。
如果在 URL 中指定了 `decompress=1` ,服务会解压通过 POST 方法发送的数据。
如果在URL中指定了`compress=1`,服务会返回压缩的数据。
如果在URL中指定了`decompress=1`服务会解压通过POST方法发送的数据。
可以通过为每份数据进行立即压缩来减少大规模数据传输中的网络压力。
您也可以选择使用[HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression)。发送一个压缩的POST请求附加请求头`Content-Encoding: compression_method`。为了使ClickHouse响应您必须附加`Accept-Encoding: compression_method`。ClickHouse支持`gzip``br`和`deflate` [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens)。要启用HTTP压缩必须使用ClickHouse[启用Http压缩](../operations/settings/settings.md#settings-enable_http_compression)配置。您可以在[Http zlib压缩级别](#settings-http_zlib_compression_level)设置中为所有压缩方法配置数据压缩级别。
可以指定 database 参数来指定默认的数据库。
您可以使用它在传输大量数据时减少网络流量,或者创建立即压缩的转储。
通过压缩发送数据的例子:
``` bash
#Sending data to the server:
$ curl -vsS "http://localhost:8123/?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' -H 'Accept-Encoding: gzip'
#Sending data to the client:
$ echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/'
```
!!! note "警告"
一些HTTP客户端可能会在默认情况下从服务器解压数据(使用`gzip`和`deflate`),即使您未正确地使用了压缩设置,您也可能会得到解压数据。
您可以使用`database`URL参数或`X-ClickHouse-Database`头来指定默认数据库。
``` bash
$ echo 'SELECT number FROM numbers LIMIT 10' | curl 'http://localhost:8123/?database=system' --data-binary @-
@ -153,30 +195,38 @@ $ echo 'SELECT number FROM numbers LIMIT 10' | curl 'http://localhost:8123/?data
9
```
默认情况下,默认数据库会在服务的配置中注册,默认是 `default`。或者,也可以在表名之前使用一个点来指定数据库。
默认情况下,在服务器设置中注册的数据库被用作默认数据库。默认情况下,它是名为`default`的数据库。或者,您可以始终在表名之前使用点来指定数据库。
用户名密码可以通过以下两种方式指定:
用户名和密码可以通过以下三种方式指定:
1. 通过 HTTP Basic Authentication。示例
1. 通过HTTP Basic Authentication。示例
<!-- -->
``` bash
echo 'SELECT 1' | curl 'http://user:password@localhost:8123/' -d @-
$ echo 'SELECT 1' | curl 'http://user:password@localhost:8123/' -d @-
```
1. 通过 URL 参数 中的 userpassword。示例:
1. 通过URL参数中的`user`和`password`。示例:
<!-- -->
``` bash
echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @-
$ echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @-
```
如果用户名没有指定,默认的用户是 `default`。如果密码没有指定,默认会使用空密码。
可以使用 URL 参数指定配置或者设置整个配置文件来处理单个查询。示例:`http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1`
1. 使用`X-ClickHouse-User`或`X-ClickHouse-Key`头指定,示例:
更多信息,参见 «[设置](../operations/settings/index.md#settings)» 部分。
<!-- -->
``` bash
$ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: password' 'http://localhost:8123/' -d @-
```
如果未指定用户名,则使用`default`。如果未指定密码,则使用空密码。
您还可以使用URL参数来指定处理单个查询或整个设置配置文件的任何设置。例子:`http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1`
更多信息,详见[设置](../operations/settings/index.md#settings)部分。
``` bash
$ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:8123/?' --data-binary @-
@ -192,30 +242,386 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812
9
```
更多关于其他参数的信息,参见 «[设置](../operations/settings/index.md#settings)» 部分
有关其他参数的信息,请参考`SET`一节
相比起 TCP 原生接口HTTP 接口不支持会话和会话设置的概念,不允许中止查询(准确地说,只在少数情况下允许),不显示查询处理的进展。执行解析和数据格式化都是在服务端处理,网络上会比 TCP 原生接口更低效
类似地您可以在HTTP协议中使用ClickHouse会话。为此需要向请求添加`session_id`GET参数。您可以使用任何字符串作为会话ID。默认情况下会话在60秒不活动后终止。要更改此超时配置请修改服务器配置中的`default_session_timeout`设置,或向请求添加`session_timeout`GET参数。要检查会话状态使用`session_check=1`参数。一次只能在单个会话中执行一个查询
可选的 `query_id` 参数可能当做 query ID 传入(或者任何字符串)。更多信息,参见 «[设置 replace_running_query](../operations/settings/settings.md)» 部分。
您可以在`X-ClickHouse-Progress`响应头中收到查询进度的信息。为此,启用[Http Header携带进度](../operations/settings/settings.md#settings-send_progress_in_http_headers)。示例:
可选的 `quota_key` 参数可能当做 quota key 传入(或者任何字符串)。更多信息,参见 «[配额](../operations/quotas.md#quotas)» 部分。
``` text
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"}
```
HTTP 接口允许传入额外的数据(外部临时表)来查询。更多信息,参见 «[外部数据查询处理](../engines/table-engines/special/external-data.md)» 部分。
显示字段信息:
## 响应缓冲 {#xiang-ying-huan-chong}
- `read_rows` — 读取的行数。
- `read_bytes` — 读取的数据字节数。
- `total_rows_to_read` — 读取的数据总行数。
- `written_rows` — 写入数据行数。
- `written_bytes` — 写入数据字节数。
可以在服务器端启用响应缓冲。提供了 `buffer_size``wait_end_of_query` 两个URL 参数来达此目的。
如果HTTP连接丢失运行的请求不会自动停止。解析和数据格式化是在服务器端执行的使用Http连接可能无效
`buffer_size` 决定了查询结果要在服务内存中缓冲多少个字节数据. 如果响应体比这个阈值大,缓冲区会写入到 HTTP 管道,剩下的数据也直接发到 HTTP 管道中。
可选的`query_id`参数可能当做query ID传入或者任何字符串。更多信息详见[replace_running_query](../operations/settings/settings.md)部分
为了确保整个响应体被缓冲,可以设置 `wait_end_of_query=1`。这种情况下,存入内存的数据会被缓冲到服务端的一个临时文件中。
可选的`quota_key`参数可能当做quota key传入或者任何字符串。更多信息详见[Quotas](../operations/quotas.md#quotas)部分。
HTTP接口允许传入额外的数据外部临时表来查询。更多信息详见[外部数据查询处理](../engines/table-engines/special/external-data.md)部分。
## 响应缓冲 {#response-buffering}
可以在服务器端启用响应缓冲。提供了`buffer_size`和`wait_end_of_query`两个URL参数来达此目的。
`buffer_size`决定了查询结果要在服务内存中缓冲多少个字节数据. 如果响应体比这个阈值大缓冲区会写入到HTTP管道剩下的数据也直接发到HTTP管道中。
为了确保整个响应体被缓冲,可以设置`wait_end_of_query=1`。这种情况下,存入内存的数据会被缓冲到服务端的一个临时文件中。
示例:
``` bash
curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wait_end_of_query=1' -d 'SELECT toUInt8(number) FROM system.numbers LIMIT 9000000 FORMAT RowBinary'
$ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wait_end_of_query=1' -d 'SELECT toUInt8(number) FROM system.numbers LIMIT 9000000 FORMAT RowBinary'
```
查询请求响应状态码和 HTTP 头被发送到客户端后,若发生查询处理出错,使用缓冲区可以避免这种情况的发生。在这种情况下,响应主体的结尾会写入一条错误消息,而在客户端,只能在解析阶段检测到该错误。
查询请求响应状态码和HTTP头被发送到客户端后若发生查询处理出错使用缓冲区可以避免这种情况的发生。在这种情况下响应主体的结尾会写入一条错误消息而在客户端只能在解析阶段检测到该错误。
### 查询参数 {#cli-queries-with-parameters}
您可以使用参数创建查询并通过相应的HTTP请求参数为它们传递值。有关更多信息请参见[CLI查询参数](../interfaces/cli.md#cli-queries-with-parameters)。
### 示例 {#example}
``` bash
$ curl -sS "<address>?param_id=2&param_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}"
```
## 特定的HTTP接口 {#predefined_http_interface}
ClickHouse通过HTTP接口支持特定的查询。例如您可以如下所示向表写入数据:
``` bash
$ echo '(4),(5),(6)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20VALUES' --data-binary @-
```
ClickHouse还支持预定义的HTTP接口可以帮助您更容易与第三方工具集成如[Prometheus Exporter](https://github.com/percona-lab/clickhouse_exporter).
示例:
- 首先,将此部分添加到服务器配置文件中:
<!-- -->
``` xml
<http_handlers>
<rule>
<url>/predefined_query</url>
<methods>POST,GET</methods>
<handler>
<type>predefined_query_handler</type>
<query>SELECT * FROM system.metrics LIMIT 5 FORMAT Template SETTINGS format_template_resultset = 'prometheus_template_output_format_resultset', format_template_row = 'prometheus_template_output_format_row', format_template_rows_between_delimiter = '\n'</query>
</handler>
</rule>
<rule>...</rule>
<rule>...</rule>
</http_handlers>
```
- 请求Prometheus格式的URL以获取数据:
<!-- -->
``` bash
$ curl -v 'http://localhost:8123/predefined_query'
* Trying ::1...
* Connected to localhost (::1) port 8123 (#0)
> GET /predefined_query HTTP/1.1
> Host: localhost:8123
> User-Agent: curl/7.47.0
> Accept: */*
>
< HTTP/1.1 200 OK
< Date: Tue, 28 Apr 2020 08:52:56 GMT
< Connection: Keep-Alive
< Content-Type: text/plain; charset=UTF-8
< X-ClickHouse-Server-Display-Name: i-mloy5trc
< Transfer-Encoding: chunked
< X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
# HELP "Query" "Number of executing queries"
# TYPE "Query" counter
"Query" 1
# HELP "Merge" "Number of executing background merges"
# TYPE "Merge" counter
"Merge" 0
# HELP "PartMutation" "Number of mutations (ALTER DELETE/UPDATE)"
# TYPE "PartMutation" counter
"PartMutation" 0
# HELP "ReplicatedFetch" "Number of data parts being fetched from replica"
# TYPE "ReplicatedFetch" counter
"ReplicatedFetch" 0
# HELP "ReplicatedSend" "Number of data parts being sent to replicas"
# TYPE "ReplicatedSend" counter
"ReplicatedSend" 0
* Connection #0 to host localhost left intact
* Connection #0 to host localhost left intact
```
正如您从示例中看到的,如果在`config.xml`文件中配置了`http_handlers`,并且`http_handlers`可以包含许多`规则`。ClickHouse将把接收到的HTTP请求与`rule`中的预定义类型进行匹配第一个匹配的将运行处理程序。如果匹配成功ClickHouse将执行相应的预定义查询。
现在`rule`可以配置`method` `header` `url` `handler`:
- `method` 负责匹配HTTP请求的方法部分。 `method`完全符合HTTP协议中[method](https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods)的定义。这是一个可选的配置。如果它没有在配置文件中定义那么它与HTTP请求的方法部分不匹配。
- `url` 负责匹配HTTP请求的URL部分。它匹配[RE2](https://github.com/google/re2)正则表达式。这是一个可选的配置。如果配置文件中没有定义它则它与HTTP请求的URL部分不匹配。
- `headers` 负责匹配HTTP请求的头部分。它与RE2的正则表达式兼容。这是一个可选的配置。如果它没有在配置文件中定义那么它与HTTP请求的头部分不匹配。
- `handler` 包含主要的处理部分。现在`handler`可以配置`type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`
`type` 目前支持三种类型:[特定查询](#predefined_query_handler), [动态查询](#dynamic_query_handler), [static](#static).
- `query` — 使用`predefined_query_handler`类型,在调用处理程序时执行查询。
- `query_param_name` — 与`dynamic_query_handler`类型一起使用提取并执行HTTP请求参数中与`query_param_name`值对应的值。
- `status` — 与`static`类型一起使用,响应状态代码。
- `content_type` — 与`static`类型一起使用,响应信息[content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type)。
- `response_content` — 与`static`类型一起使用,响应发送给客户端的内容,当使用前缀`file://`或`config://`时,从发送给客户端的文件或配置中查找内容。
接下来是不同`type`的配置方法。
### 特定查询 {#predefined_query_handler}
`predefined_query_handler` 支持设置`Settings`和`query_params`参数。您可以将`query`配置为`predefined_query_handler`类型。
`query` 是一个预定义的`predefined_query_handler`查询它由ClickHouse在匹配HTTP请求并返回查询结果时执行。这是一个必须的配置。
以下是定义的[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_alter_threads`设置, 然后查询系统表以检查这些设置是否设置成功。
示例:
``` xml
<http_handlers>
<rule>
<url><![CDATA[/query_param_with_url/\w+/(?P<name_1>[^/]+)(/(?P<name_2>[^/]+))?]]></url>
<method>GET</method>
<headers>
<XXX>TEST_HEADER_VALUE</XXX>
<PARAMS_XXX><![CDATA[(?P<name_1>[^/]+)(/(?P<name_2>[^/]+))?]]></PARAMS_XXX>
</headers>
<handler>
<type>predefined_query_handler</type>
<query>SELECT value FROM system.settings WHERE name = {name_1:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {name_2:String}</query>
</handler>
</rule>
</http_handlers>
```
``` bash
$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2'
1
max_alter_threads 2
```
!!! note "警告"
在一个`predefined_query_handler`中只支持insert类型的一个`查询`。
### 动态查询 {#dynamic_query_handler}
`dynamic_query_handler`查询以HTTP请求参数的形式编写。区别在于在`predefined_query_handler`中,查询是在配置文件中编写的。您可以在`dynamic_query_handler`中配置`query_param_name`。
ClickHouse提取并执行与HTTP请求URL中的`query_param_name`值对应的值。`query_param_name`的默认值是`/query`。这是一个可选的配置。如果配置文件中没有定义,则不会传入参数。
为了试验这个功能,示例定义了[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_alter_threads``queries`设置是否成功的值。
示例:
``` xml
<http_handlers>
<rule>
<headers>
<XXX>TEST_HEADER_VALUE_DYNAMIC</XXX> </headers>
<handler>
<type>dynamic_query_handler</type>
<query_param_name>query_param</query_param_name>
</handler>
</rule>
</http_handlers>
```
``` bash
$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2&param_name_1=max_threads&param_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D'
max_threads 1
max_alter_threads 2
```
### static {#static}
`static`可以返回[content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [status](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status)和`response_content`。`response_content`可以返回指定的内容。
示例:
返回信息.
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/hi</url>
<handler>
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>Say Hi!</response_content>
</handler>
</rule>
<http_handlers>
```
``` bash
$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
* Trying ::1...
* Connected to localhost (::1) port 8123 (#0)
> GET /hi HTTP/1.1
> Host: localhost:8123
> User-Agent: curl/7.47.0
> Accept: */*
> XXX:xxx
>
< HTTP/1.1 402 Payment Required
< Date: Wed, 29 Apr 2020 03:51:26 GMT
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
* Connection #0 to host localhost left intact
Say Hi!%
```
从配置中查找发送到客户端的内容。
``` xml
<get_config_static_handler><![CDATA[<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>]]></get_config_static_handler>
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/get_config_static_handler</url>
<handler>
<type>static</type>
<response_content>config://get_config_static_handler</response_content>
</handler>
</rule>
</http_handlers>
```
``` bash
$ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
* Trying ::1...
* Connected to localhost (::1) port 8123 (#0)
> GET /get_config_static_handler HTTP/1.1
> Host: localhost:8123
> User-Agent: curl/7.47.0
> Accept: */*
> XXX:xxx
>
< HTTP/1.1 200 OK
< Date: Wed, 29 Apr 2020 04:01:24 GMT
< Connection: Keep-Alive
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
* Connection #0 to host localhost left intact
<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>%
```
从发送到客户端的文件中查找内容。
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/get_absolute_path_static_handler</url>
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</rule>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/get_relative_path_static_handler</url>
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</rule>
</http_handlers>
```
``` bash
$ user_files_path='/var/lib/clickhouse/user_files'
$ sudo echo "<html><body>Relative Path File</body></html>" > $user_files_path/relative_path_file.html
$ sudo echo "<html><body>Absolute Path File</body></html>" > $user_files_path/absolute_path_file.html
$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
* Trying ::1...
* Connected to localhost (::1) port 8123 (#0)
> GET /get_absolute_path_static_handler HTTP/1.1
> Host: localhost:8123
> User-Agent: curl/7.47.0
> Accept: */*
> XXX:xxx
>
< HTTP/1.1 200 OK
< Date: Wed, 29 Apr 2020 04:18:16 GMT
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
<html><body>Absolute Path File</body></html>
* Connection #0 to host localhost left intact
$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
* Trying ::1...
* Connected to localhost (::1) port 8123 (#0)
> GET /get_relative_path_static_handler HTTP/1.1
> Host: localhost:8123
> User-Agent: curl/7.47.0
> Accept: */*
> XXX:xxx
>
< HTTP/1.1 200 OK
< Date: Wed, 29 Apr 2020 04:18:31 GMT
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
<html><body>Relative Path File</body></html>
* Connection #0 to host localhost left intact
```
[来源文章](https://clickhouse.tech/docs/zh/interfaces/http_interface/) <!--hide-->

View File

@ -1,19 +1,27 @@
---
toc_folder_title: Interfaces
toc_priority: 14
toc_title: 客户端
---
# 客户端 {#interfaces}
ClickHouse提供了两个网络接口两者都可以选择包装在TLS中以提高安全性
ClickHouse提供了两个网络接口(两个都可以选择包装在TLS中以增加安全性):
- [HTTP](http.md),记录在案,易于使用.
- [本地TCP](tcp.md),这有较少的开销.
- [HTTP](http.md), 包含文档,易于使用。
- [Native TCP](../interfaces/tcp.md),简单,方便使用。
在大多数情况下,建议使用适当的工具或库,而不是直接与这些工具或库进行交互。 Yandex的官方支持如下
\* [命令行客户端](cli.md)
\* [JDBC驱动程序](jdbc.md)
\* [ODBC驱动程序](odbc.md)
\* [C++客户端库](cpp.md)
在大多数情况下建议使用适当的工具或库而不是直接与它们交互。Yandex官方支持的项目有:
还有许多第三方库可供使用ClickHouse
\* [客户端库](third-party/client-libraries.md)
\* [集成](third-party/integrations.md)
\* [可视界面](third-party/gui.md)
- [命令行客户端](../interfaces/cli.md)
- [JDBC驱动](../interfaces/jdbc.md)
- [ODBC驱动](../interfaces/odbc.md)
- [C++客户端](../interfaces/cpp.md)
[来源文章](https://clickhouse.tech/docs/zh/interfaces/) <!--hide-->
还有一些广泛的第三方库可供ClickHouse使用:
- [客户端库](../interfaces/third-party/client-libraries.md)
- [第三方集成库](../interfaces/third-party/integrations.md)
- [可视化UI](../interfaces/third-party/gui.md)
[来源文章](https://clickhouse.tech/docs/en/interfaces/) <!--hide-->

View File

@ -1,8 +1,13 @@
# JDBC 驱动 {#jdbc-qu-dong}
---
toc_priority: 22
toc_title: JDBC驱动
---
- **[官方JDBC 的驱动](https://github.com/ClickHouse/clickhouse-jdbc)**
- 三方提供的 JDBC 驱动:
- [掳胫–禄脢鹿脷露胫鲁隆鹿–酶](https://github.com/housepower/ClickHouse-Native-JDBC)
# JDBC驱动 {#jdbc-driver}
- **[官方驱动](https://github.com/ClickHouse/clickhouse-jdbc)**
- 第三方驱动:
- [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC)
- [clickhouse4j](https://github.com/blynkkk/clickhouse4j)
[来源文章](https://clickhouse.tech/docs/zh/interfaces/jdbc/) <!--hide-->

View File

@ -1,19 +1,17 @@
---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 20
toc_title: "MySQL\u63A5\u53E3"
toc_title: MySQL接口
---
# MySQL接口 {#mysql-interface}
ClickHouse支持MySQL线协议。 它可以通过启用 [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) 在配置文件中设置:
ClickHouse支持MySQL协议。它可以通过[mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port)在配置文件中设置:
``` xml
<mysql_port>9004</mysql_port>
```
使用命令行工具连接的示例 `mysql`:
使用命令行工具连接`mysql`示例:
``` bash
$ mysql --protocol tcp -u default -P 9004
@ -37,8 +35,8 @@ Type 'help;' or '\h' for help. Type '\c' to clear the current input statement.
mysql>
```
为了与所有MySQL客户端兼容建议使用以下命令指定用户密码 [双SHA1](../operations/settings/settings-users.md#password_double_sha1_hex) 在配置文件中
如果使用用户密码指定 [SHA256](../operations/settings/settings-users.md#password_sha256_hex)一些客户端将无法进行身份验证mysqljs和旧版本的命令行工具mysql
为了与所有MySQL客户端兼容建议在配置文件中使用[双SHA1](../operations/settings/settings-users.md#password_double_sha1_hex)指定用户密码
如果使用[SHA256](../operations/settings/settings-users.md#password_sha256_hex)指定用户密码,一些客户端将无法进行身份验证(mysqljs和旧版本的命令行工具mysql)
限制:

View File

@ -1,5 +1,10 @@
# ODBC 驱动 {#odbc-qu-dong}
---
toc_priority: 23
toc_title: ODBC驱动
---
- ClickHouse官方有 ODBC 的驱动。 见 [这里](https://github.com/ClickHouse/clickhouse-odbc)。
# ODBC驱动 {#odbc-driver}
- [官方驱动](https://github.com/ClickHouse/clickhouse-odbc)。
[来源文章](https://clickhouse.tech/docs/zh/interfaces/odbc/) <!--hide-->

View File

@ -1,5 +1,10 @@
# 原生客户端接口TCP {#yuan-sheng-ke-hu-duan-jie-kou-tcp}
---
toc_priority: 18
toc_title: 原生接口(TCP)
---
本机协议用于 [命令行客户端](cli.md)用于分布式查询处理期间的服务器间通信以及其他C ++程序。 不幸的是本机ClickHouse协议还没有正式的规范但它可以从ClickHouse源代码进行逆向工程 [从这里开始](https://github.com/ClickHouse/ClickHouse/tree/master/src/Client))和/或拦截和分析TCP流量。
# 原生接口TCP{#native-interface-tcp}
原生接口用于[命令行客户端](cli.md)用于分布式查询处理期间的服务器间通信以及其他C++程序。可惜的是原生的ClickHouse协议还没有正式的规范但它可以从ClickHouse[源代码](https://github.com/ClickHouse/ClickHouse/tree/master/src/Client)通过拦截和分析TCP流量进行反向工程。
[来源文章](https://clickhouse.tech/docs/zh/interfaces/tcp/) <!--hide-->

View File

@ -1,13 +1,108 @@
# 条件函数 {#tiao-jian-han-shu}
## 如果cond那么否则cond 运算符然后else {#ifcond-then-else-cond-operator-then-else}
## if {#if}
控制条件分支。 与大多数系统不同ClickHouse始终评估两个表达式 `then``else`
**语法**
``` sql
SELECT if(cond, then, else)
```
如果条件 `cond` 的计算结果为非零值,则返回表达式 `then` 的结果,并且跳过表达式 `else` 的结果(如果存在)。 如果 `cond` 为零或 `NULL`,则将跳过 `then` 表达式的结果,并返回 `else` 表达式的结果(如果存在)。
**参数**
- `cond` 条件结果可以为零或不为零。 类型是 UInt8Nullable(UInt8) 或 NULL。
- `then` - 如果满足条件则返回的表达式。
- `else` - 如果不满足条件则返回的表达式。
**返回值**
该函数执行 `then``else` 表达式并返回其结果,这取决于条件 `cond` 最终是否为零。
**示例**
查询:
``` sql
SELECT if(1, plus(2, 2), plus(2, 6))
```
结果:
``` text
┌─plus(2, 2)─┐
│ 4 │
└────────────┘
```
查询:
``` sql
SELECT if(0, plus(2, 2), plus(2, 6))
```
结果:
``` text
┌─plus(2, 6)─┐
│ 8 │
└────────────┘
```
- `then``else` 必须具有最低的通用类型。
**示例:**
给定表`LEFT_RIGHT`:
``` sql
SELECT *
FROM LEFT_RIGHT
┌─left─┬─right─┐
│ ᴺᵁᴸᴸ │ 4 │
│ 1 │ 3 │
│ 2 │ 2 │
│ 3 │ 1 │
│ 4 │ ᴺᵁᴸᴸ │
└──────┴───────┘
```
下面的查询比较了 `left``right` 的值:
``` sql
SELECT
left,
right,
if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller
FROM LEFT_RIGHT
WHERE isNotNull(left) AND isNotNull(right)
┌─left─┬─right─┬─is_smaller──────────────────────────┐
│ 1 │ 3 │ left is smaller than right │
│ 2 │ 2 │ right is greater or equal than left │
│ 3 │ 1 │ right is greater or equal than left │
└──────┴───────┴─────────────────────────────────────┘
```
注意:在此示例中未使用'NULL'值,请检查[条件中的NULL值](#null-values-in-conditionals) 部分。
## 三元运算符 {#ternary-operator}
`if` 函数相同。
语法: `cond ? then : else`
如果`cond = 0`则返回`then`,如果`cond = 0`则返回`else`。
`cond`必须是`UInt8`类型,`then`和`else`必须存在最低的共同类型。
`then`和`else`可以是`NULL`
- `cond`必须是`UInt8`类型,`then`和`else`必须存在最低的共同类型。
## 多 {#multiif}
- `then`和`else`可以是`NULL`
## multiIf {#multiif}
允许您在查询中更紧凑地编写[CASE](../operators/index.md#operator_case)运算符。
@ -27,18 +122,74 @@
**示例**
存在如下一张表
再次使用表 `LEFT_RIGHT`
┌─x─┬────y─┐
│ 1 │ ᴺᵁᴸᴸ │
│ 2 │ 3 │
└───┴──────┘
``` sql
SELECT
left,
right,
multiIf(left < right, 'left is smaller', left > right, 'left is greater', left = right, 'Both equal', 'Null value') AS result
FROM LEFT_RIGHT
执行查询 `SELECT multiIf(isNull(y), x, y < 3, y, NULL) FROM t_null`。结果:
┌─left─┬─right─┬─result──────────┐
│ ᴺᵁᴸᴸ │ 4 │ Null value │
│ 1 │ 3 │ left is smaller │
│ 2 │ 2 │ Both equal │
│ 3 │ 1 │ left is greater │
│ 4 │ ᴺᵁᴸᴸ │ Null value │
└──────┴───────┴─────────────────┘
```
## 直接使用条件结果 {#using-conditional-results-directly}
┌─multiIf(isNull(y), x, less(y, 3), y, NULL)─┐
│ 1 │
│ ᴺᵁᴸᴸ │
└────────────────────────────────────────────┘
条件结果始终为 `0``1``NULL`。 因此,你可以像这样直接使用条件结果:
``` sql
SELECT left < right AS is_small
FROM LEFT_RIGHT
┌─is_small─┐
│ ᴺᵁᴸᴸ │
│ 1 │
│ 0 │
│ 0 │
│ ᴺᵁᴸᴸ │
└──────────┘
```
## 条件中的NULL值 {#null-values-in-conditionals}
当条件中包含 `NULL` 值时,结果也将为 `NULL`
``` sql
SELECT
NULL < 1,
2 < NULL,
NULL < NULL,
NULL = NULL
┌─less(NULL, 1)─┬─less(2, NULL)─┬─less(NULL, NULL)─┬─equals(NULL, NULL)─┐
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │
└───────────────┴───────────────┴──────────────────┴────────────────────┘
```
因此,如果类型是 `Nullable`,你应该仔细构造查询。
以下示例说明这一点。
``` sql
SELECT
left,
right,
multiIf(left < right, 'left is smaller', left > right, 'right is smaller', 'Both equal') AS faulty_result
FROM LEFT_RIGHT
┌─left─┬─right─┬─faulty_result────┐
│ ᴺᵁᴸᴸ │ 4 │ Both equal │
│ 1 │ 3 │ left is smaller │
│ 2 │ 2 │ Both equal │
│ 3 │ 1 │ right is smaller │
│ 4 │ ᴺᵁᴸᴸ │ Both equal │
└──────┴───────┴──────────────────┘
```
[来源文章](https://clickhouse.tech/docs/en/query_language/functions/conditional_functions/) <!--hide-->

View File

@ -1,5 +1,71 @@
# 编码函数 {#bian-ma-han-shu}
## char {#char}
返回长度为传递参数数量的字符串并且每个字节都有对应参数的值。接受数字Numeric类型的多个参数。如果参数的值超出了UInt8数据类型的范围则将其转换为UInt8并可能进行舍入和溢出。
**语法**
``` sql
char(number_1, [number_2, ..., number_n]);
```
**参数**
- `number_1, number_2, ..., number_n` — 数值参数解释为整数。类型: [Int](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md).
**返回值**
- 给定字节数的字符串。
类型: `String`
**示例**
查询:
``` sql
SELECT char(104.1, 101, 108.9, 108.9, 111) AS hello
```
结果:
``` text
┌─hello─┐
│ hello │
└───────┘
```
你可以通过传递相应的字节来构造任意编码的字符串。 这是UTF-8的示例:
查询:
``` sql
SELECT char(0xD0, 0xBF, 0xD1, 0x80, 0xD0, 0xB8, 0xD0, 0xB2, 0xD0, 0xB5, 0xD1, 0x82) AS hello;
```
结果:
``` text
┌─hello──┐
│ привет │
└────────┘
```
查询:
``` sql
SELECT char(0xE4, 0xBD, 0xA0, 0xE5, 0xA5, 0xBD) AS hello;
```
结果:
``` text
┌─hello─┐
│ 你好 │
└───────┘
```
## hex {#hex}
接受`String``unsigned integer``Date`或`DateTime`类型的参数。返回包含参数的十六进制表示的字符串。使用大写字母`A-F`。不使用`0x`前缀或`h`后缀。对于字符串,所有字节都简单地编码为两个十六进制数字。数字转换为大端(«易阅读»)格式。对于数字,去除其中较旧的零,但仅限整个字节。例如,`hex1='01'`。 `Date`被编码为自Unix时间开始以来的天数。 `DateTime`编码为自Unix时间开始以来的秒数。
@ -17,11 +83,11 @@
接受FixedString16值。返回包含36个字符的文本格式的字符串。
## 位掩码列表(num) {#bitmasktolistnum}
## bitmaskToList(num) {#bitmasktolistnum}
接受一个整数。返回一个字符串其中包含一组2的幂列表其列表中的所有值相加等于这个整数。列表使用逗号分割按升序排列。
## 位掩码阵列(num) {#bitmasktoarraynum}
## bitmaskToArray(num) {#bitmasktoarraynum}
接受一个整数。返回一个UInt64类型数组其中包含一组2的幂列表其列表中的所有值相加等于这个整数。数组中的数字按升序排列。

View File

@ -6,7 +6,7 @@
您可以向它传递任何类型的参数,但传递的参数将不会使用在任何随机数生成过程中。
此参数的唯一目的是防止公共子表达式消除,以便在相同的查询中使用相同的随机函数生成不同的随机数。
## 兰德 {#rand}
## rand, rand32 {#rand}
返回一个UInt32类型的随机数字所有UInt32类型的数字被生成的概率均相等。此函数线性同于的方式生成随机数。

View File

@ -1,6 +1,6 @@
# 字符串函数 {#zi-fu-chuan-han-shu}
## {#string-functions-empty}
## empty {#string-functions-empty}
对于空字符串返回1对于非空字符串返回0。
结果类型是UInt8。
@ -13,13 +13,13 @@
结果类型是UInt8。
该函数也适用于数组。
## 长度 {#length}
## length {#length}
返回字符串的字节长度。
结果类型是UInt64。
该函数也适用于数组。
## 长度8 {#lengthutf8}
## lengthUTF8 {#lengthutf8}
假定字符串以UTF-8编码组成的文本返回此字符串的Unicode字符长度。如果传入的字符串不是UTF-8编码则函数可能返回一个预期外的值不会抛出异常
结果类型是UInt64。
@ -29,16 +29,16 @@
假定字符串以UTF-8编码组成的文本返回此字符串的Unicode字符长度。如果传入的字符串不是UTF-8编码则函数可能返回一个预期外的值不会抛出异常
结果类型是UInt64。
## 字符长度,字符长度 {#character-length-character-length}
## character_length,CHARACTER_LENGTH {#character-length-character-length}
假定字符串以UTF-8编码组成的文本返回此字符串的Unicode字符长度。如果传入的字符串不是UTF-8编码则函数可能返回一个预期外的值不会抛出异常
结果类型是UInt64。
## 低一点 {#lower-lcase}
## lower, lcase {#lower-lcase}
将字符串中的ASCII转换为小写。
## 上,ucase {#upper-ucase}
## upper, ucase {#upper-ucase}
将字符串中的ASCII转换为大写。
@ -84,7 +84,7 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b')
└───────────────────────┘
```
## 反向 {#reverse}
## reverse {#reverse}
反转字符串。
@ -118,11 +118,11 @@ SELECT format('{} {}', 'Hello', 'World')
与[concat](#concat-s1-s2)相同区别在于你需要保证concat(s1, s2, s3) -\> s4是单射的它将用于GROUP BY的优化。
## 子串(s,offset,length),mid(s,offset,length),substr(s,offset,length) {#substrings-offset-length-mids-offset-length-substrs-offset-length}
## substring(s,offset,length),mid(s,offset,length),substr(s,offset,length) {#substrings-offset-length-mids-offset-length-substrs-offset-length}
以字节为单位截取指定位置字符串返回以offset位置为开头长度为length的子串。offset从1开始与标准SQL相同offsetlength参数必须是常量。
## substringf8(s,offset,length) {#substringutf8s-offset-length}
## substringUTF8(s,offset,length) {#substringutf8s-offset-length}
substring相同但其操作单位为Unicode字符函数假设字符串是以UTF-8进行编码的文本。如果不是则可能返回一个预期外的结果不会抛出异常
@ -150,7 +150,7 @@ SELECT format('{} {}', 'Hello', 'World')
返回是否以指定的后缀结尾。如果字符串以指定的后缀结束则返回1否则返回0。
## 开始使用s前缀) {#startswiths-prefix}
## startsWiths前缀) {#startswiths-prefix}
返回是否以指定的前缀开头。如果字符串以指定的前缀开头则返回1否则返回0。

View File

@ -151,7 +151,7 @@ DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER
删除配额。
已删除的配额将从分配配额的所有实体撤销。
已删除的配额将从分配配额的所有实体撤销。
### 语法 {#drop-quota-syntax}
@ -161,9 +161,9 @@ DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
## DROP SETTINGS PROFILE {#drop-settings-profile-statement}
删除配额
删除settings配置
已删除的配额将从分配配额的所有实体撤销。
已删除的settings配置将从分配该settings配置的所有实体撤销。
### 语法 {#drop-settings-profile-syntax}
@ -177,7 +177,7 @@ DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
EXISTS [TEMPORARY] [TABLE|DICTIONARY] [db.]name [INTO OUTFILE filename] [FORMAT format]
```
返回单 `UInt8`-type column,其中包含单个值 `0` 如果表或数据库不存在,或 `1` 如果该表存在于指定的数据库中。
返回单`UInt8` 类型的列,其中包含单个值 `0` 如果表或数据库不存在,或 `1` 如果该表存在于指定的数据库中。
## KILL QUERY {#kill-query-statement}

View File

@ -60,13 +60,13 @@ public:
const String & user_, const String & password_, const String & stage,
bool randomize_, size_t max_iterations_, double max_time_,
const String & json_path_, size_t confidence_,
const String & query_id_, bool continue_on_errors_,
const String & query_id_, const String & query_to_execute_, bool continue_on_errors_,
bool print_stacktrace_, const Settings & settings_)
:
concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), confidence(confidence_), query_id(query_id_),
continue_on_errors(continue_on_errors_),
query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_),
print_stacktrace(print_stacktrace_), settings(settings_),
shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())),
pool(concurrency)
@ -150,7 +150,8 @@ private:
double max_time;
String json_path;
size_t confidence;
std::string query_id;
String query_id;
String query_to_execute;
bool continue_on_errors;
bool print_stacktrace;
const Settings & settings;
@ -213,20 +214,28 @@ private:
void readQueries()
{
ReadBufferFromFileDescriptor in(STDIN_FILENO);
while (!in.eof())
if (query_to_execute.empty())
{
std::string query;
readText(query, in);
assertChar('\n', in);
ReadBufferFromFileDescriptor in(STDIN_FILENO);
if (!query.empty())
queries.emplace_back(query);
while (!in.eof())
{
String query;
readText(query, in);
assertChar('\n', in);
if (!query.empty())
queries.emplace_back(std::move(query));
}
if (queries.empty())
throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED);
}
else
{
queries.emplace_back(query_to_execute);
}
if (queries.empty())
throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED);
std::cerr << "Loaded " << queries.size() << " queries.\n";
}
@ -559,6 +568,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth());
desc.add_options()
("help", "produce help message")
("query", value<std::string>()->default_value(""), "query to execute")
("concurrency,c", value<unsigned>()->default_value(1), "number of parallel queries")
("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)")
("stage", value<std::string>()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation")
@ -625,6 +635,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["json"].as<std::string>(),
options["confidence"].as<size_t>(),
options["query_id"].as<std::string>(),
options["query"].as<std::string>(),
options.count("continue_on_errors") > 0,
print_stacktrace,
settings);

View File

@ -951,7 +951,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new TCPHandlerFactory(*this),
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false),
server_pool,
socket,
new Poco::Net::TCPServerParams));
@ -959,6 +959,22 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString());
});
/// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt
create_server("tcp_with_proxy_port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true),
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with native protocol (tcp) with PROXY: {}", address.toString());
});
/// TCP with SSL
create_server("tcp_port_secure", [&](UInt16 port)
{
@ -968,7 +984,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new TCPHandlerFactory(*this, /* secure= */ true),
new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false),
server_pool,
socket,
new Poco::Net::TCPServerParams));

View File

@ -0,0 +1 @@
../../../tests/config/config.d/tcp_with_proxy.xml

View File

@ -64,11 +64,18 @@
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<!-- For HTTPS and SSL over native protocol. -->
<!--
<https_port>8443</https_port>
<tcp_port_secure>9440</tcp_port_secure>
-->
<!-- TCP with PROXY protocol (PROXY header sent for every connection) -->
<!--
<tcp_with_proxy_port>9010</tcp_with_proxy_port>
-->
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->

View File

@ -352,17 +352,20 @@
/// The query is saved in browser history (in state JSON object)
/// as well as in URL fragment identifier.
if (query != previous_query) {
previous_query = query;
var state = {
query: query,
status: this.status,
response: this.response.length > 100000 ? null : this.response /// Lower than the browser's limit.
};
var title = "ClickHouse Query: " + query;
history.pushState(
{
query: query,
status: this.status,
response: this.response.length > 100000 ? null : this.response /// Lower than the browser's limit.
},
title,
window.location.pathname + '?user=' + encodeURIComponent(user) + '#' + window.btoa(query));
var url = window.location.pathname + '?user=' + encodeURIComponent(user) + '#' + window.btoa(query);
if (previous_query == '') {
history.replaceState(state, title, url);
} else {
history.pushState(state, title, url);
}
document.title = title;
previous_query = query;
}
} else {
//console.log(this);

View File

@ -15,6 +15,8 @@
#include <common/find_symbols.h>
#include <Poco/ExpireCache.h>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <filesystem>
#include <mutex>
@ -419,6 +421,18 @@ std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(
params.http_method = client_info.http_method;
params.address = client_info.current_address.host();
params.quota_key = client_info.quota_key;
/// Extract the last entry from comma separated list of X-Forwarded-For addresses.
/// Only the last proxy can be trusted (if any).
Strings forwarded_addresses;
boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(","));
if (!forwarded_addresses.empty())
{
String & last_forwarded_address = forwarded_addresses.back();
boost::trim(last_forwarded_address);
params.forwarded_address = last_forwarded_address;
}
return getContextAccess(params);
}
@ -444,9 +458,14 @@ std::shared_ptr<const EnabledRowPolicies> AccessControlManager::getEnabledRowPol
std::shared_ptr<const EnabledQuota> AccessControlManager::getEnabledQuota(
const UUID & user_id, const String & user_name, const boost::container::flat_set<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const
const UUID & user_id,
const String & user_name,
const boost::container::flat_set<UUID> & enabled_roles,
const Poco::Net::IPAddress & address,
const String & forwarded_address,
const String & custom_quota_key) const
{
return quota_cache->getEnabledQuota(user_id, user_name, enabled_roles, address, custom_quota_key);
return quota_cache->getEnabledQuota(user_id, user_name, enabled_roles, address, forwarded_address, custom_quota_key);
}

View File

@ -135,6 +135,7 @@ public:
const String & user_name,
const boost::container::flat_set<UUID> & enabled_roles,
const Poco::Net::IPAddress & address,
const String & forwarded_address,
const String & custom_quota_key) const;
std::vector<QuotaUsage> getAllQuotasUsage() const;

View File

@ -127,6 +127,7 @@ enum class AccessType
M(SYSTEM_DROP_COMPILED_EXPRESSION_CACHE, "SYSTEM DROP COMPILED EXPRESSION, DROP COMPILED EXPRESSION CACHE, DROP COMPILED EXPRESSIONS", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \
M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \
M(SYSTEM_RELOAD_SYMBOLS, "RELOAD SYMBOLS", GLOBAL, SYSTEM_RELOAD) \
M(SYSTEM_RELOAD_DICTIONARY, "SYSTEM RELOAD DICTIONARIES, RELOAD DICTIONARY, RELOAD DICTIONARIES", GLOBAL, SYSTEM_RELOAD) \
M(SYSTEM_RELOAD_EMBEDDED_DICTIONARIES, "RELOAD EMBEDDED DICTIONARIES", GLOBAL, SYSTEM_RELOAD) /* implicitly enabled by the grant SYSTEM_RELOAD_DICTIONARY ON *.* */\
M(SYSTEM_RELOAD, "", GROUP, SYSTEM) \

View File

@ -258,9 +258,12 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
{
assert(roles_info_);
roles_info = roles_info_;
enabled_row_policies = manager->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles);
enabled_quota = manager->getEnabledQuota(*params.user_id, user_name, roles_info->enabled_roles, params.address, params.quota_key);
enabled_settings = manager->getEnabledSettings(*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles);
enabled_row_policies = manager->getEnabledRowPolicies(
*params.user_id, roles_info->enabled_roles);
enabled_quota = manager->getEnabledQuota(
*params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key);
enabled_settings = manager->getEnabledSettings(
*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles);
calculateAccessRights();
}

View File

@ -41,9 +41,16 @@ struct ContextAccessParams
ClientInfo::Interface interface = ClientInfo::Interface::TCP;
ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN;
Poco::Net::IPAddress address;
String forwarded_address;
String quota_key;
auto toTuple() const { return std::tie(user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection, current_database, interface, http_method, address, quota_key); }
auto toTuple() const
{
return std::tie(
user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection,
current_database, interface, http_method, address, forwarded_address, quota_key);
}
friend bool operator ==(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs == rhs); }
friend bool operator <(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() < rhs.toTuple(); }

View File

@ -25,9 +25,10 @@ public:
String user_name;
boost::container::flat_set<UUID> enabled_roles;
Poco::Net::IPAddress client_address;
String forwarded_address;
String client_key;
auto toTuple() const { return std::tie(user_id, enabled_roles, user_name, client_address, client_key); }
auto toTuple() const { return std::tie(user_id, enabled_roles, user_name, client_address, forwarded_address, client_key); }
friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); }

View File

@ -76,6 +76,7 @@ struct Quota : public IAccessEntity
NONE, /// All users share the same quota.
USER_NAME, /// Connections with the same user name share the same quota.
IP_ADDRESS, /// Connections from the same IP share the same quota.
FORWARDED_IP_ADDRESS, /// Use X-Forwarded-For HTTP header instead of IP address.
CLIENT_KEY, /// Client should explicitly supply a key to use.
CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key.
CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key.
@ -205,12 +206,16 @@ inline const Quota::KeyTypeInfo & Quota::KeyTypeInfo::get(KeyType type)
if (tokens.size() > 1)
{
for (const auto & token : tokens)
{
for (auto kt : ext::range(KeyType::MAX))
{
if (KeyTypeInfo::get(kt).name == token)
{
init_base_types.push_back(kt);
break;
}
}
}
}
return KeyTypeInfo{raw_name_, std::move(init_name), std::move(init_base_types)};
};
@ -232,6 +237,11 @@ inline const Quota::KeyTypeInfo & Quota::KeyTypeInfo::get(KeyType type)
static const auto info = make_info("IP_ADDRESS");
return info;
}
case KeyType::FORWARDED_IP_ADDRESS:
{
static const auto info = make_info("FORWARDED_IP_ADDRESS");
return info;
}
case KeyType::CLIENT_KEY:
{
static const auto info = make_info("CLIENT_KEY");

View File

@ -48,11 +48,21 @@ String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
switch (quota->key_type)
{
case KeyType::NONE:
{
return "";
}
case KeyType::USER_NAME:
{
return params.user_name;
}
case KeyType::IP_ADDRESS:
{
return params.client_address.toString();
}
case KeyType::FORWARDED_IP_ADDRESS:
{
return params.forwarded_address;
}
case KeyType::CLIENT_KEY:
{
if (!params.client_key.empty())
@ -170,7 +180,7 @@ QuotaCache::QuotaCache(const AccessControlManager & access_control_manager_)
QuotaCache::~QuotaCache() = default;
std::shared_ptr<const EnabledQuota> QuotaCache::getEnabledQuota(const UUID & user_id, const String & user_name, const boost::container::flat_set<UUID> & enabled_roles, const Poco::Net::IPAddress & client_address, const String & client_key)
std::shared_ptr<const EnabledQuota> QuotaCache::getEnabledQuota(const UUID & user_id, const String & user_name, const boost::container::flat_set<UUID> & enabled_roles, const Poco::Net::IPAddress & client_address, const String & forwarded_address, const String & client_key)
{
std::lock_guard lock{mutex};
ensureAllQuotasRead();
@ -180,6 +190,7 @@ std::shared_ptr<const EnabledQuota> QuotaCache::getEnabledQuota(const UUID & use
params.user_name = user_name;
params.enabled_roles = enabled_roles;
params.client_address = client_address;
params.forwarded_address = forwarded_address;
params.client_key = client_key;
auto it = enabled_quotas.find(params);
if (it != enabled_quotas.end())

View File

@ -20,7 +20,14 @@ public:
QuotaCache(const AccessControlManager & access_control_manager_);
~QuotaCache();
std::shared_ptr<const EnabledQuota> getEnabledQuota(const UUID & user_id, const String & user_name, const boost::container::flat_set<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key);
std::shared_ptr<const EnabledQuota> getEnabledQuota(
const UUID & user_id,
const String & user_name,
const boost::container::flat_set<UUID> & enabled_roles,
const Poco::Net::IPAddress & address,
const String & forwarded_address,
const String & client_key);
std::vector<QuotaUsage> getAllQuotasUsage() const;
private:

View File

@ -215,6 +215,8 @@ namespace
String quota_config = "quotas." + quota_name;
if (config.has(quota_config + ".keyed_by_ip"))
quota->key_type = KeyType::IP_ADDRESS;
else if (config.has(quota_config + ".keyed_by_forwarded_ip"))
quota->key_type = KeyType::FORWARDED_IP_ADDRESS;
else if (config.has(quota_config + ".keyed"))
quota->key_type = KeyType::CLIENT_KEY_OR_USER_NAME;
else

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include "registerAggregateFunctions.h"

View File

@ -100,6 +100,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, c
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
//if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
}

View File

@ -237,6 +237,8 @@ template <typename T, typename TResult, typename Data, AggregateFunctionSumType
class AggregateFunctionSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data, Type>>
{
public:
static constexpr bool DateTime64Supported = false;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<TResult>, DataTypeNumber<TResult>>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, ColumnVector<TResult>>;

View File

@ -135,6 +135,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal256) return new AggregateFunctionTemplate<Decimal256>(std::forward<TArgs>(args)...);
if constexpr (AggregateFunctionTemplate<DateTime64>::DateTime64Supported)
if (which.idx == TypeIndex::DateTime64) return new AggregateFunctionTemplate<DateTime64>(std::forward<TArgs>(args)...);
return nullptr;
}
@ -146,6 +148,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64, Data>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128, Data>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal256) return new AggregateFunctionTemplate<Decimal256, Data>(std::forward<TArgs>(args)...);
if constexpr (AggregateFunctionTemplate<DateTime64, Data>::DateTime64Supported)
if (which.idx == TypeIndex::DateTime64) return new AggregateFunctionTemplate<DateTime64, Data>(std::forward<TArgs>(args)...);
return nullptr;
}

View File

@ -310,6 +310,9 @@ protected:
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
public:
// Derived class can `override` this to flag that DateTime64 is not supported.
static constexpr bool DateTime64Supported = true;
IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}

View File

@ -1,8 +1,8 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Common/typeid_cast.h>
#include <Core/Defines.h>
namespace DB
@ -25,6 +25,13 @@ Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const
for (size_t i = 0; i < parameters.size(); ++i)
{
const auto * literal = parameters[i]->as<ASTLiteral>();
ASTPtr func_literal;
if (!literal)
if (const auto * func = parameters[i]->as<ASTFunction>())
if ((func_literal = func->toLiteral()))
literal = func_literal->as<ASTLiteral>();
if (!literal)
{
throw Exception(

View File

@ -6,18 +6,6 @@ if (USE_CLANG_TIDY)
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
endif ()
if(COMPILER_PIPE)
set(MAX_COMPILER_MEMORY 2500)
else()
set(MAX_COMPILER_MEMORY 1500)
endif()
if(MAKE_STATIC_LIBRARIES)
set(MAX_LINKER_MEMORY 3500)
else()
set(MAX_LINKER_MEMORY 2500)
endif()
include(../cmake/limit_jobs.cmake)
set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h)
set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/Common/config.h)
@ -49,6 +37,7 @@ add_subdirectory (Dictionaries)
add_subdirectory (Disks)
add_subdirectory (Storages)
add_subdirectory (Parsers)
add_subdirectory (Parsers/New)
add_subdirectory (IO)
add_subdirectory (Functions)
add_subdirectory (Interpreters)
@ -186,12 +175,12 @@ endif()
if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES)
add_library (dbms STATIC ${dbms_headers} ${dbms_sources})
target_link_libraries (dbms PRIVATE jemalloc libdivide ${DBMS_COMMON_LIBRARIES})
target_link_libraries (dbms PRIVATE clickhouse_parsers_new jemalloc libdivide ${DBMS_COMMON_LIBRARIES})
set (all_modules dbms)
else()
add_library (dbms SHARED ${dbms_headers} ${dbms_sources})
target_link_libraries (dbms PUBLIC ${all_modules} ${DBMS_COMMON_LIBRARIES})
target_link_libraries (clickhouse_interpreters PRIVATE jemalloc libdivide)
target_link_libraries (clickhouse_interpreters PRIVATE clickhouse_parsers_new jemalloc libdivide)
list (APPEND all_modules dbms)
# force all split libs to be linked
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-as-needed")
@ -253,7 +242,7 @@ target_link_libraries (clickhouse_common_io
PUBLIC
common
${DOUBLE_CONVERSION_LIBRARIES}
ryu
dragonbox_to_chars
)
if(RE2_LIBRARY)
@ -329,6 +318,8 @@ dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR})
if (ZSTD_LIBRARY)
dbms_target_link_libraries(PRIVATE ${ZSTD_LIBRARY})
target_link_libraries (clickhouse_common_io PUBLIC ${ZSTD_LIBRARY})
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ZSTD_INCLUDE_DIR})
if (NOT USE_INTERNAL_ZSTD_LIBRARY AND ZSTD_INCLUDE_DIR)
dbms_target_include_directories(SYSTEM BEFORE PRIVATE ${ZSTD_INCLUDE_DIR})
endif ()

View File

@ -370,4 +370,5 @@ template class ColumnDecimal<Decimal32>;
template class ColumnDecimal<Decimal64>;
template class ColumnDecimal<Decimal128>;
template class ColumnDecimal<Decimal256>;
template class ColumnDecimal<DateTime64>;
}

View File

@ -526,6 +526,8 @@
M(557, UNKNOWN_UNION) \
M(558, EXPECTED_ALL_OR_DISTINCT) \
M(559, INVALID_GRPC_QUERY_INFO) \
M(560, ZSTD_ENCODER_FAILED) \
M(561, ZSTD_DECODER_FAILED) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

Some files were not shown because too many files have changed in this diff Show More