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

* upstream/master:
  Add support agreement page and snippets. (#34512)
  Fixed unit tests
  Fixed tests
  For SQLUserDefinedFunctions change access type from DATABASE to GLOBAL
  Fix test
  avoid unnecessary copying of Settings
  Fix possible error 'file_size: Operation not supported'
  Replace clickhouse_grpc.proto in a test with symlink.
  Rename QueryInfo's field result_compression -> transport_compression_type and change its type for better consistency. Make the previous field obsolete.
  gRPC: Split compression_type field into input_compression_type and output_compression_type. Make the previous field obsolete.
  clang-tidy reported divide by zero exception
This commit is contained in:
cnmade 2022-02-11 12:11:10 +08:00
commit ba91e5130a
29 changed files with 373 additions and 323 deletions

View File

@ -217,13 +217,12 @@
<!-- The following file is used only if ssl_require_client_auth=1 -->
<ssl_ca_cert_file>/path/to/ssl_ca_cert_file</ssl_ca_cert_file>
<!-- Default compression algorithm (applied if client doesn't specify another algorithm, see result_compression in QueryInfo).
<!-- Default transport compression type (can be overridden by client, see the transport_compression_type field in QueryInfo).
Supported algorithms: none, deflate, gzip, stream_gzip -->
<compression>deflate</compression>
<transport_compression_type>none</transport_compression_type>
<!-- Default compression level (applied if client doesn't specify another level, see result_compression in QueryInfo).
Supported levels: none, low, medium, high -->
<compression_level>medium</compression_level>
<!-- Default transport compression level. Supported levels: 0..3 -->
<transport_compression_level>0</transport_compression_level>
<!-- Send/receive message size limits in bytes. -1 means unlimited -->
<max_send_message_size>-1</max_send_message_size>

View File

@ -86,7 +86,7 @@ enum class AccessType
M(CREATE_DICTIONARY, "", DICTIONARY, CREATE) /* allows to execute {CREATE|ATTACH} DICTIONARY */\
M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables;
implicitly enabled by the grant CREATE_TABLE on any table */ \
M(CREATE_FUNCTION, "", DATABASE, CREATE) /* allows to execute CREATE FUNCTION */ \
M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \
M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \
\
M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\
@ -94,7 +94,7 @@ enum class AccessType
M(DROP_VIEW, "", VIEW, DROP) /* allows to execute {DROP|DETACH} TABLE for views;
implicitly enabled by the grant DROP_TABLE */\
M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\
M(DROP_FUNCTION, "", DATABASE, DROP) /* allows to execute DROP FUNCTION */\
M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\
M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\
\
M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \

View File

@ -425,6 +425,7 @@ bool ContextAccess::checkAccessImplHelper(const AccessFlags & flags, const Args
| AccessType::TRUNCATE;
const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY;
const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION;
const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl;
const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE;
const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS;
@ -432,7 +433,7 @@ bool ContextAccess::checkAccessImplHelper(const AccessFlags & flags, const Args
const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY;
const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE;
const AccessFlags ddl_flags = table_ddl | dictionary_ddl;
const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl;
const AccessFlags introspection_flags = AccessType::INTROSPECTION;
};
static const PrecalculatedFlags precalc;

View File

@ -45,7 +45,7 @@ TEST(AccessRights, Union)
lhs.grant(AccessType::INSERT);
rhs.grant(AccessType::ALL, "db1");
lhs.makeUnion(rhs);
ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, CREATE FUNCTION, DROP, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
}

View File

@ -239,6 +239,7 @@ private:
UInt64 genRandom(size_t lim)
{
assert(lim > 0);
/// With a large number of values, we will generate random numbers several times slower.
if (lim <= static_cast<UInt64>(rng.max()))
return static_cast<UInt32>(rng()) % static_cast<UInt32>(lim);

View File

@ -1317,7 +1317,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (insert && insert->select)
insert->tryFindInputFunction(input_function);
bool is_async_insert = global_context->getSettings().async_insert && insert && insert->hasInlinedData();
bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData();
/// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately.
if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert)

View File

@ -316,7 +316,7 @@ getTableOutput(const String & database_name, const String & table_name, ContextM
return std::move(res.pipeline);
}
static inline String reWriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, const String & database_name, const String & table_name, const Settings & global_settings)
static inline String rewriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, const String & database_name, const String & table_name, const Settings & global_settings)
{
Block tables_columns_sample_block
{
@ -376,7 +376,7 @@ static inline void dumpDataForTables(
auto pipeline = getTableOutput(database_name, table_name, query_context);
StreamSettings mysql_input_stream_settings(context->getSettingsRef());
String mysql_select_all_query = "SELECT " + reWriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettings()) + " FROM "
String mysql_select_all_query = "SELECT " + rewriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettingsRef()) + " FROM "
+ backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name);
LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "mysql_select_all_query is {}", mysql_select_all_query);
auto input = std::make_unique<MySQLSource>(connection, mysql_select_all_query, pipeline.getHeader(), mysql_input_stream_settings);

View File

@ -197,7 +197,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
size_t max_command_execution_time = config.getUInt64(settings_config_prefix + ".max_command_execution_time", 10);
size_t max_execution_time_seconds = static_cast<size_t>(context->getSettings().max_execution_time.totalSeconds());
size_t max_execution_time_seconds = static_cast<size_t>(context->getSettingsRef().max_execution_time.totalSeconds());
if (max_execution_time_seconds != 0 && max_command_execution_time > max_execution_time_seconds)
max_command_execution_time = max_execution_time_seconds;

View File

@ -229,7 +229,7 @@ ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info)
bool secure = cluster_info.current_node.secure;
auto cluster = std::make_shared<Cluster>(
context->getSettings(),
context->getSettingsRef(),
shards,
/* username= */ context->getUserName(),
/* password= */ "",

View File

@ -921,8 +921,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
auto reader_settings = getMergeTreeReaderSettings(context);
bool use_skip_indexes = context->getSettings().use_skip_indexes;
if (select.final() && !context->getSettings().use_skip_indexes_if_final)
bool use_skip_indexes = settings.use_skip_indexes;
if (select.final() && !settings.use_skip_indexes_if_final)
use_skip_indexes = false;
result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(

View File

@ -59,7 +59,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery(
: std::make_unique<EmptyReadBuffer>();
/// Create a source from input buffer using format from query
auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettings().max_insert_block_size);
auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettingsRef().max_insert_block_size);
source->addBuffer(std::move(input_buffer));
return source;
}

View File

@ -51,6 +51,7 @@ using GRPCQueryInfo = clickhouse::grpc::QueryInfo;
using GRPCResult = clickhouse::grpc::Result;
using GRPCException = clickhouse::grpc::Exception;
using GRPCProgress = clickhouse::grpc::Progress;
using GRPCObsoleteTransportCompression = clickhouse::grpc::ObsoleteTransportCompression;
namespace DB
{
@ -101,62 +102,6 @@ namespace
});
}
grpc_compression_algorithm parseCompressionAlgorithm(const String & str)
{
if (str == "none")
return GRPC_COMPRESS_NONE;
else if (str == "deflate")
return GRPC_COMPRESS_DEFLATE;
else if (str == "gzip")
return GRPC_COMPRESS_GZIP;
else if (str == "stream_gzip")
return GRPC_COMPRESS_STREAM_GZIP;
else
throw Exception("Unknown compression algorithm: '" + str + "'", ErrorCodes::INVALID_CONFIG_PARAMETER);
}
grpc_compression_level parseCompressionLevel(const String & str)
{
if (str == "none")
return GRPC_COMPRESS_LEVEL_NONE;
else if (str == "low")
return GRPC_COMPRESS_LEVEL_LOW;
else if (str == "medium")
return GRPC_COMPRESS_LEVEL_MED;
else if (str == "high")
return GRPC_COMPRESS_LEVEL_HIGH;
else
throw Exception("Unknown compression level: '" + str + "'", ErrorCodes::INVALID_CONFIG_PARAMETER);
}
grpc_compression_algorithm convertCompressionAlgorithm(const ::clickhouse::grpc::CompressionAlgorithm & algorithm)
{
if (algorithm == ::clickhouse::grpc::NO_COMPRESSION)
return GRPC_COMPRESS_NONE;
else if (algorithm == ::clickhouse::grpc::DEFLATE)
return GRPC_COMPRESS_DEFLATE;
else if (algorithm == ::clickhouse::grpc::GZIP)
return GRPC_COMPRESS_GZIP;
else if (algorithm == ::clickhouse::grpc::STREAM_GZIP)
return GRPC_COMPRESS_STREAM_GZIP;
else
throw Exception("Unknown compression algorithm: '" + ::clickhouse::grpc::CompressionAlgorithm_Name(algorithm) + "'", ErrorCodes::INVALID_GRPC_QUERY_INFO);
}
grpc_compression_level convertCompressionLevel(const ::clickhouse::grpc::CompressionLevel & level)
{
if (level == ::clickhouse::grpc::COMPRESSION_NONE)
return GRPC_COMPRESS_LEVEL_NONE;
else if (level == ::clickhouse::grpc::COMPRESSION_LOW)
return GRPC_COMPRESS_LEVEL_LOW;
else if (level == ::clickhouse::grpc::COMPRESSION_MEDIUM)
return GRPC_COMPRESS_LEVEL_MED;
else if (level == ::clickhouse::grpc::COMPRESSION_HIGH)
return GRPC_COMPRESS_LEVEL_HIGH;
else
throw Exception("Unknown compression level: '" + ::clickhouse::grpc::CompressionLevel_Name(level) + "'", ErrorCodes::INVALID_GRPC_QUERY_INFO);
}
/// Gets file's contents as a string, throws an exception if failed.
String readFile(const String & filepath)
{
@ -193,6 +138,102 @@ namespace
return grpc::InsecureServerCredentials();
}
/// Transport compression makes gRPC library to compress packed Result messages before sending them through network.
struct TransportCompression
{
grpc_compression_algorithm algorithm;
grpc_compression_level level;
/// Extracts the settings of transport compression from a query info if possible.
static std::optional<TransportCompression> fromQueryInfo(const GRPCQueryInfo & query_info)
{
TransportCompression res;
if (!query_info.transport_compression_type().empty())
{
res.setAlgorithm(query_info.transport_compression_type(), ErrorCodes::INVALID_GRPC_QUERY_INFO);
res.setLevel(query_info.transport_compression_level(), ErrorCodes::INVALID_GRPC_QUERY_INFO);
return res;
}
if (query_info.has_obsolete_result_compression())
{
switch (query_info.obsolete_result_compression().algorithm())
{
case GRPCObsoleteTransportCompression::NO_COMPRESSION: res.algorithm = GRPC_COMPRESS_NONE; break;
case GRPCObsoleteTransportCompression::DEFLATE: res.algorithm = GRPC_COMPRESS_DEFLATE; break;
case GRPCObsoleteTransportCompression::GZIP: res.algorithm = GRPC_COMPRESS_GZIP; break;
case GRPCObsoleteTransportCompression::STREAM_GZIP: res.algorithm = GRPC_COMPRESS_STREAM_GZIP; break;
default: throw Exception(ErrorCodes::INVALID_GRPC_QUERY_INFO, "Unknown compression algorithm: {}", GRPCObsoleteTransportCompression::CompressionAlgorithm_Name(query_info.obsolete_result_compression().algorithm()));
}
switch (query_info.obsolete_result_compression().level())
{
case GRPCObsoleteTransportCompression::COMPRESSION_NONE: res.level = GRPC_COMPRESS_LEVEL_NONE; break;
case GRPCObsoleteTransportCompression::COMPRESSION_LOW: res.level = GRPC_COMPRESS_LEVEL_LOW; break;
case GRPCObsoleteTransportCompression::COMPRESSION_MEDIUM: res.level = GRPC_COMPRESS_LEVEL_MED; break;
case GRPCObsoleteTransportCompression::COMPRESSION_HIGH: res.level = GRPC_COMPRESS_LEVEL_HIGH; break;
default: throw Exception(ErrorCodes::INVALID_GRPC_QUERY_INFO, "Unknown compression level: {}", GRPCObsoleteTransportCompression::CompressionLevel_Name(query_info.obsolete_result_compression().level()));
}
return res;
}
return std::nullopt;
}
/// Extracts the settings of transport compression from the server configuration.
static TransportCompression fromConfiguration(const Poco::Util::AbstractConfiguration & config)
{
TransportCompression res;
if (config.has("grpc.transport_compression_type"))
{
res.setAlgorithm(config.getString("grpc.transport_compression_type"), ErrorCodes::INVALID_CONFIG_PARAMETER);
res.setLevel(config.getInt("grpc.transport_compression_level", 0), ErrorCodes::INVALID_CONFIG_PARAMETER);
}
else
{
res.setAlgorithm(config.getString("grpc.compression", "none"), ErrorCodes::INVALID_CONFIG_PARAMETER);
res.setLevel(config.getString("grpc.compression_level", "none"), ErrorCodes::INVALID_CONFIG_PARAMETER);
}
return res;
}
private:
void setAlgorithm(const String & str, int error_code)
{
if (str == "none")
algorithm = GRPC_COMPRESS_NONE;
else if (str == "deflate")
algorithm = GRPC_COMPRESS_DEFLATE;
else if (str == "gzip")
algorithm = GRPC_COMPRESS_GZIP;
else if (str == "stream_gzip")
algorithm = GRPC_COMPRESS_STREAM_GZIP;
else
throw Exception(error_code, "Unknown compression algorithm: '{}'", str);
}
void setLevel(const String & str, int error_code)
{
if (str == "none")
level = GRPC_COMPRESS_LEVEL_NONE;
else if (str == "low")
level = GRPC_COMPRESS_LEVEL_LOW;
else if (str == "medium")
level = GRPC_COMPRESS_LEVEL_MED;
else if (str == "high")
level = GRPC_COMPRESS_LEVEL_HIGH;
else
throw Exception(error_code, "Unknown compression level: '{}'", str);
}
void setLevel(int level_, int error_code)
{
if (0 <= level_ && level_ < GRPC_COMPRESS_LEVEL_COUNT)
level = static_cast<grpc_compression_level>(level_);
else
throw Exception(error_code, "Compression level {} is out of range 0..{}", level_, GRPC_COMPRESS_LEVEL_COUNT - 1);
}
};
/// Gets session's timeout from query info or from the server config.
std::chrono::steady_clock::duration getSessionTimeout(const GRPCQueryInfo & query_info, const Poco::Util::AbstractConfiguration & config)
@ -293,15 +334,10 @@ namespace
return std::nullopt;
}
void setResultCompression(grpc_compression_algorithm algorithm, grpc_compression_level level)
void setTransportCompression(const TransportCompression & transport_compression)
{
grpc_context.set_compression_algorithm(algorithm);
grpc_context.set_compression_level(level);
}
void setResultCompression(const ::clickhouse::grpc::Compression & compression)
{
setResultCompression(convertCompressionAlgorithm(compression.algorithm()), convertCompressionLevel(compression.level()));
grpc_context.set_compression_algorithm(transport_compression.algorithm);
grpc_context.set_compression_level(transport_compression.level);
}
protected:
@ -628,10 +664,11 @@ namespace
ASTInsertQuery * insert_query = nullptr;
String input_format;
String input_data_delimiter;
CompressionMethod input_compression_method = CompressionMethod::None;
PODArray<char> output;
String output_format;
CompressionMethod compression_method = CompressionMethod::None;
int compression_level = 0;
CompressionMethod output_compression_method = CompressionMethod::None;
int output_compression_level = 0;
uint64_t interactive_delay = 100000;
bool send_exception_with_stacktrace = true;
@ -815,9 +852,9 @@ namespace
if (!query_info.database().empty())
query_context->setCurrentDatabase(query_info.database());
/// Apply compression settings for this call.
if (query_info.has_result_compression())
responder->setResultCompression(query_info.result_compression());
/// Apply transport compression for this call.
if (auto transport_compression = TransportCompression::fromQueryInfo(query_info))
responder->setTransportCompression(*transport_compression);
/// The interactive delay will be used to show progress.
interactive_delay = settings.interactive_delay;
@ -852,8 +889,16 @@ namespace
output_format = query_context->getDefaultFormat();
/// Choose compression.
compression_method = chooseCompressionMethod("", query_info.compression_type());
compression_level = query_info.compression_level();
String input_compression_method_str = query_info.input_compression_type();
if (input_compression_method_str.empty())
input_compression_method_str = query_info.obsolete_compression_type();
input_compression_method = chooseCompressionMethod("", input_compression_method_str);
String output_compression_method_str = query_info.output_compression_type();
if (output_compression_method_str.empty())
output_compression_method_str = query_info.obsolete_compression_type();
output_compression_method = chooseCompressionMethod("", output_compression_method_str);
output_compression_level = query_info.output_compression_level();
/// Set callback to create and fill external tables
query_context->setExternalTablesInitializer([this] (ContextPtr context)
@ -984,7 +1029,7 @@ namespace
return {nullptr, 0}; /// no more input data
});
read_buffer = wrapReadBufferWithCompressionMethod(std::move(read_buffer), compression_method);
read_buffer = wrapReadBufferWithCompressionMethod(std::move(read_buffer), input_compression_method);
assert(!pipeline);
auto source = query_context->getInputFormat(
@ -1112,13 +1157,13 @@ namespace
if (io.pipeline.pulling())
header = io.pipeline.getHeader();
if (compression_method != CompressionMethod::None)
if (output_compression_method != CompressionMethod::None)
output.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data.
write_buffer = std::make_unique<WriteBufferFromVector<PODArray<char>>>(output);
nested_write_buffer = static_cast<WriteBufferFromVector<PODArray<char>> *>(write_buffer.get());
if (compression_method != CompressionMethod::None)
if (output_compression_method != CompressionMethod::None)
{
write_buffer = wrapWriteBufferWithCompressionMethod(std::move(write_buffer), compression_method, compression_level);
write_buffer = wrapWriteBufferWithCompressionMethod(std::move(write_buffer), output_compression_method, output_compression_level);
compressing_write_buffer = write_buffer.get();
}
@ -1414,10 +1459,10 @@ namespace
return;
PODArray<char> memory;
if (compression_method != CompressionMethod::None)
if (output_compression_method != CompressionMethod::None)
memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data.
std::unique_ptr<WriteBuffer> buf = std::make_unique<WriteBufferFromVector<PODArray<char>>>(memory);
buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level);
buf = wrapWriteBufferWithCompressionMethod(std::move(buf), output_compression_method, output_compression_level);
auto format = query_context->getOutputFormat(output_format, *buf, totals);
format->write(materializeBlock(totals));
format->finalize();
@ -1432,10 +1477,10 @@ namespace
return;
PODArray<char> memory;
if (compression_method != CompressionMethod::None)
if (output_compression_method != CompressionMethod::None)
memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data.
std::unique_ptr<WriteBuffer> buf = std::make_unique<WriteBufferFromVector<PODArray<char>>>(memory);
buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level);
buf = wrapWriteBufferWithCompressionMethod(std::move(buf), output_compression_method, output_compression_level);
auto format = query_context->getOutputFormat(output_format, *buf, extremes);
format->write(materializeBlock(extremes));
format->finalize();
@ -1772,8 +1817,9 @@ void GRPCServer::start()
builder.RegisterService(&grpc_service);
builder.SetMaxSendMessageSize(iserver.config().getInt("grpc.max_send_message_size", -1));
builder.SetMaxReceiveMessageSize(iserver.config().getInt("grpc.max_receive_message_size", -1));
builder.SetDefaultCompressionAlgorithm(parseCompressionAlgorithm(iserver.config().getString("grpc.compression", "none")));
builder.SetDefaultCompressionLevel(parseCompressionLevel(iserver.config().getString("grpc.compression_level", "none")));
auto default_transport_compression = TransportCompression::fromConfiguration(iserver.config());
builder.SetDefaultCompressionAlgorithm(default_transport_compression.algorithm);
builder.SetDefaultCompressionLevel(default_transport_compression.level);
queue = builder.AddCompletionQueue();
grpc_server = builder.BuildAndStart();

View File

@ -36,7 +36,7 @@ void HTTPServerConnection::run()
if (request.isSecure())
{
size_t hsts_max_age = context->getSettings().hsts_max_age.value;
size_t hsts_max_age = context->getSettingsRef().hsts_max_age.value;
if (hsts_max_age > 0)
response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age));

View File

@ -45,21 +45,19 @@ message ExternalTable {
map<string, string> settings = 5;
}
enum CompressionAlgorithm {
message ObsoleteTransportCompression {
enum CompressionAlgorithm {
NO_COMPRESSION = 0;
DEFLATE = 1;
GZIP = 2;
STREAM_GZIP = 3;
}
enum CompressionLevel {
}
enum CompressionLevel {
COMPRESSION_NONE = 0;
COMPRESSION_LOW = 1;
COMPRESSION_MEDIUM = 2;
COMPRESSION_HIGH = 3;
}
message Compression {
}
CompressionAlgorithm algorithm = 1;
CompressionLevel level = 2;
}
@ -102,16 +100,16 @@ message QueryInfo {
// `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used.
bool next_query_info = 16;
/// Controls how a ClickHouse server will compress query execution results before sending back to the client.
/// If not set the compression settings from the configuration file will be used.
Compression result_compression = 17;
// Compression type for `input_data`, `output_data`, `totals` and `extremes`.
// Compression type for `input_data`.
// Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2.
// When used for `input_data` the client is responsible to compress data before putting it into `input_data`.
// When used for `output_data` or `totals` or `extremes` the client receives compressed data and should decompress it by itself.
// In the latter case consider to specify also `compression_level`.
string compression_type = 18;
// The client is responsible to compress data before putting it into `input_data`.
string input_compression_type = 20;
// Compression type for `output_data`, `totals` and `extremes`.
// Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2.
// The client receives compressed data and should decompress it by itself.
// Consider also setting `output_compression_level`.
string output_compression_type = 21;
// Compression level.
// WARNING: If it's not specified the compression level is set to zero by default which might be not the best choice for some compression types (see below).
@ -123,7 +121,23 @@ message QueryInfo {
// zstd: 1..22; 3 is recommended by default (compression level 0 also means 3)
// lz4: 0..16; values < 0 mean fast acceleration
// bz2: 1..9
int32 compression_level = 19;
int32 output_compression_level = 19;
// Transport compression is an alternative way to make the server to compress its response.
// This kind of compression implies that instead of compressing just `output` the server will compress whole packed messages of the `Result` type,
// and then gRPC implementation on client side will decompress those messages so client code won't be bothered with decompression.
// Here is a big difference between the transport compression and the compression enabled by setting `output_compression_type` because
// in case of the transport compression the client code receives already decompressed data in `output`.
// If the transport compression is not set here it can still be enabled by the server configuration.
// Supported compression types: none, deflate, gzip, stream_gzip
// Supported compression levels: 0..3
// WARNING: Don't set `transport_compression` and `output_compression` at the same time because it will make the server to compress its output twice!
string transport_compression_type = 22;
int32 transport_compression_level = 23;
/// Obsolete fields, should not be used in new code.
ObsoleteTransportCompression obsolete_result_compression = 17;
string obsolete_compression_type = 18;
}
enum LogsLevel {

View File

@ -69,7 +69,7 @@ Pipe StorageHDFSCluster::read(
size_t /*max_block_size*/,
unsigned /*num_streams*/)
{
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings());
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef());
auto iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(context, uri);
auto callback = std::make_shared<HDFSSource::IteratorWrapper>([iterator]() mutable -> String

View File

@ -879,9 +879,10 @@ SinkToStoragePtr StorageFile::write(
path = paths.back();
fs::create_directories(fs::path(path).parent_path());
std::error_code error_code;
if (!context->getSettingsRef().engine_file_truncate_on_insert && !is_path_with_globs
&& !FormatFactory::instance().checkIfFormatSupportAppend(format_name, context, format_settings) && fs::exists(paths.back())
&& fs::file_size(paths.back()) != 0)
&& fs::file_size(paths.back(), error_code) != 0 && !error_code)
{
if (context->getSettingsRef().engine_file_allow_create_multiple_files)
{

View File

@ -82,7 +82,7 @@ Pipe StorageS3Cluster::read(
{
StorageS3::updateClientAndAuthSettings(context, client_auth);
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings());
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef());
StorageS3::updateClientAndAuthSettings(context, client_auth);
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);

View File

@ -66,7 +66,7 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat
}
StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context)
: query_id(context->getCurrentQueryId()), settings(context->getSettings())
: query_id(context->getCurrentQueryId()), settings(context->getSettingsRef())
{
/// Will apply WHERE to subset of columns and then add more columns.
/// This is kind of complicated, but we use WHERE to do less work.

View File

@ -205,7 +205,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
if (name != "clusterAllReplicas")
cluster = context->getCluster(cluster_name_expanded);
else
cluster = context->getCluster(cluster_name_expanded)->getClusterWithReplicasAsShards(context->getSettings());
cluster = context->getCluster(cluster_name_expanded)->getClusterWithReplicasAsShards(context->getSettingsRef());
}
else
{
@ -241,7 +241,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
bool treat_local_as_remote = false;
bool treat_local_port_as_remote = context->getApplicationType() == Context::ApplicationType::LOCAL;
cluster = std::make_shared<Cluster>(
context->getSettings(),
context->getSettingsRef(),
names,
configuration.username,
configuration.password,

View File

@ -373,22 +373,14 @@ def test_cancel_while_generating_output():
output += result.output
assert output == b'0\t0\n1\t0\n2\t0\n3\t0\n'
def test_result_compression():
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000000)",
result_compression=clickhouse_grpc_pb2.Compression(algorithm=clickhouse_grpc_pb2.CompressionAlgorithm.GZIP,
level=clickhouse_grpc_pb2.CompressionLevel.COMPRESSION_HIGH))
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
result = stub.ExecuteQuery(query_info)
assert result.output == (b'0\n')*1000000
def test_compressed_output():
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="lz4")
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", output_compression_type="lz4")
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
result = stub.ExecuteQuery(query_info)
assert lz4.frame.decompress(result.output) == (b'0\n')*1000
def test_compressed_output_streaming():
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(100000)", compression_type="lz4")
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(100000)", output_compression_type="lz4")
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
d_context = lz4.frame.create_decompression_context()
data = b''
@ -398,7 +390,7 @@ def test_compressed_output_streaming():
assert data == (b'0\n')*100000
def test_compressed_output_gzip():
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="gzip", compression_level=6)
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", output_compression_type="gzip", output_compression_level=6)
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
result = stub.ExecuteQuery(query_info)
assert gzip.decompress(result.output) == (b'0\n')*1000
@ -407,10 +399,10 @@ def test_compressed_totals_and_extremes():
query("CREATE TABLE t (x UInt8, y UInt8) ENGINE = Memory")
query("INSERT INTO t VALUES (1, 2), (2, 4), (3, 2), (3, 3), (3, 4)")
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT sum(x), y FROM t GROUP BY y WITH TOTALS", compression_type="lz4")
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT sum(x), y FROM t GROUP BY y WITH TOTALS", output_compression_type="lz4")
result = stub.ExecuteQuery(query_info)
assert lz4.frame.decompress(result.totals) == b'12\t0\n'
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT x, y FROM t", settings={"extremes": "1"}, compression_type="lz4")
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT x, y FROM t", settings={"extremes": "1"}, output_compression_type="lz4")
result = stub.ExecuteQuery(query_info)
assert lz4.frame.decompress(result.extremes) == b'1\t2\n3\t4\n'
@ -423,7 +415,7 @@ def test_compressed_insert_query_streaming():
d2 = data[sz1:sz1+sz2]
d3 = data[sz1+sz2:]
def send_query_info():
yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t VALUES", input_data=d1, compression_type="lz4", next_query_info=True)
yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t VALUES", input_data=d1, input_compression_type="lz4", next_query_info=True)
yield clickhouse_grpc_pb2.QueryInfo(input_data=d2, next_query_info=True)
yield clickhouse_grpc_pb2.QueryInfo(input_data=d3)
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
@ -445,6 +437,12 @@ def test_compressed_external_table():
b"4\tDaniel\n"\
b"5\tEthan\n"
def test_transport_compression():
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000000)", transport_compression_type='gzip', transport_compression_level=3)
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
result = stub.ExecuteQuery(query_info)
assert result.output == (b'0\n')*1000000
def test_opentelemetry_context_propagation():
trace_id = "80c190b5-9dc1-4eae-82b9-6c261438c817"
parent_span_id = 123

View File

@ -1,174 +0,0 @@
/* This file describes gRPC protocol supported in ClickHouse.
*
* To use this protocol a client should send one or more messages of the QueryInfo type
* and then receive one or more messages of the Result type.
* According to that the service provides four methods for that:
* ExecuteQuery(QueryInfo) returns (Result)
* ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result)
* ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result)
* ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result)
* It's up to the client to choose which method to use.
* For example, ExecuteQueryWithStreamInput() allows the client to add data multiple times
* while executing a query, which is suitable for inserting many rows.
*/
syntax = "proto3";
package clickhouse.grpc;
message NameAndType {
string name = 1;
string type = 2;
}
// Describes an external table - a table which will exists only while a query is executing.
message ExternalTable {
// Name of the table. If omitted, "_data" is used.
string name = 1;
// Columns of the table. Types are required, names can be omitted. If the names are omitted, "_1", "_2", ... is used.
repeated NameAndType columns = 2;
// Data to insert to the external table.
// If a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used,
// then data for insertion to the same external table can be split between multiple QueryInfos.
bytes data = 3;
// Format of the data to insert to the external table.
string format = 4;
// Settings for executing that insertion, applied after QueryInfo.settings.
map<string, string> settings = 5;
}
enum CompressionAlgorithm {
NO_COMPRESSION = 0;
DEFLATE = 1;
GZIP = 2;
STREAM_GZIP = 3;
}
enum CompressionLevel {
COMPRESSION_NONE = 0;
COMPRESSION_LOW = 1;
COMPRESSION_MEDIUM = 2;
COMPRESSION_HIGH = 3;
}
message Compression {
CompressionAlgorithm algorithm = 1;
CompressionLevel level = 2;
}
// Information about a query which a client sends to a ClickHouse server.
// The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data.
// In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set.
message QueryInfo {
string query = 1;
string query_id = 2;
map<string, string> settings = 3;
// Default database.
string database = 4;
// Input data, used both as data for INSERT query and as data for the input() function.
bytes input_data = 5;
// Delimiter for input_data, inserted between input_data from adjacent QueryInfos.
bytes input_data_delimiter = 6;
// Default output format. If not specified, 'TabSeparated' is used.
string output_format = 7;
repeated ExternalTable external_tables = 8;
string user_name = 9;
string password = 10;
string quota = 11;
// Works exactly like sessions in the HTTP protocol.
string session_id = 12;
bool session_check = 13;
uint32 session_timeout = 14;
// Set `cancel` to true to stop executing the query.
bool cancel = 15;
// If true there will be at least one more QueryInfo in the input stream.
// `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used.
bool next_query_info = 16;
/// Controls how a ClickHouse server will compress query execution results before sending back to the client.
/// If not set the compression settings from the configuration file will be used.
Compression result_compression = 17;
}
enum LogsLevel {
LOG_NONE = 0;
LOG_FATAL = 1;
LOG_CRITICAL = 2;
LOG_ERROR = 3;
LOG_WARNING = 4;
LOG_NOTICE = 5;
LOG_INFORMATION = 6;
LOG_DEBUG = 7;
LOG_TRACE = 8;
}
message LogEntry {
uint32 time = 1;
uint32 time_microseconds = 2;
uint64 thread_id = 3;
string query_id = 4;
LogsLevel level = 5;
string source = 6;
string text = 7;
}
message Progress {
uint64 read_rows = 1;
uint64 read_bytes = 2;
uint64 total_rows_to_read = 3;
uint64 written_rows = 4;
uint64 written_bytes = 5;
}
message Stats {
uint64 rows = 1;
uint64 blocks = 2;
uint64 allocated_bytes = 3;
bool applied_limit = 4;
uint64 rows_before_limit = 5;
}
message Exception {
int32 code = 1;
string name = 2;
string display_text = 3;
string stack_trace = 4;
}
// Result of execution of a query which is sent back by the ClickHouse server to the client.
message Result {
// Output of the query, represented in the `output_format` or in a format specified in `query`.
bytes output = 1;
bytes totals = 2;
bytes extremes = 3;
repeated LogEntry logs = 4;
Progress progress = 5;
Stats stats = 6;
// Set by the ClickHouse server if there was an exception thrown while executing.
Exception exception = 7;
// Set by the ClickHouse server if executing was cancelled by the `cancel` field in QueryInfo.
bool cancelled = 8;
}
service ClickHouse {
rpc ExecuteQuery(QueryInfo) returns (Result) {}
rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {}
rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {}
rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {}
}

View File

@ -0,0 +1 @@
../../../../src/Server/grpc_protos/clickhouse_grpc.proto

View File

@ -49,8 +49,8 @@
<create_query>CREATE TABLE IF NOT EXISTS table_{format_slow} ENGINE = File({format_slow}, '/dev/null') AS test.hits</create_query>
<create_query>CREATE TABLE IF NOT EXISTS table_{format_fast} ENGINE = File({format_fast}, '/dev/null') AS test.hits</create_query>
<query>INSERT INTO table_{format_slow} SELECT * FROM test.hits LIMIT 10000 SETTINGS engine_file_truncate_on_insert = 1</query>
<query>INSERT INTO table_{format_fast} SELECT * FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert = 1</query>
<query>INSERT INTO table_{format_slow} SELECT * FROM test.hits LIMIT 10000</query>
<query>INSERT INTO table_{format_fast} SELECT * FROM test.hits LIMIT 100000</query>
<drop_query>DROP TABLE IF EXISTS table_{format_slow}</drop_query>
<drop_query>DROP TABLE IF EXISTS table_{format_fast}</drop_query>

View File

@ -20,7 +20,7 @@
<create_query>CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}, '/dev/null') AS SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits limit 0</create_query>
<query>INSERT INTO table_{format} SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert = 1</query>
<query>INSERT INTO table_{format} SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits LIMIT 100000</query>
<drop_query>DROP TABLE IF EXISTS table_{format}</drop_query>
</test>

View File

@ -49,13 +49,13 @@ CREATE TABLE [] TABLE CREATE
CREATE VIEW [] VIEW CREATE
CREATE DICTIONARY [] DICTIONARY CREATE
CREATE TEMPORARY TABLE [] GLOBAL CREATE
CREATE FUNCTION [] DATABASE CREATE
CREATE FUNCTION [] GLOBAL CREATE
CREATE [] \N ALL
DROP DATABASE [] DATABASE DROP
DROP TABLE [] TABLE DROP
DROP VIEW [] VIEW DROP
DROP DICTIONARY [] DICTIONARY DROP
DROP FUNCTION [] DATABASE DROP
DROP FUNCTION [] GLOBAL DROP
DROP [] \N ALL
TRUNCATE ['TRUNCATE TABLE'] TABLE ALL
OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL

View File

@ -0,0 +1,6 @@
-- Tags: no-fasttest
insert into table function file('/dev/null', 'Parquet', 'number UInt64') select * from numbers(10);
insert into table function file('/dev/null', 'ORC', 'number UInt64') select * from numbers(10);
insert into table function file('/dev/null', 'JSON', 'number UInt64') select * from numbers(10);

View File

@ -0,0 +1,27 @@
{% set prefetch_items = [
('/docs/en/', 'document')
] %}
{% extends "templates/base.html" %}
{% block extra_meta %}
<meta name="robots" content="noindex">
{% include "templates/common_fonts.html" %}
{% endblock %}
{% block nav %}
{% include "templates/global/nav.html" %}
{% endblock %}
{% block content %}
{% include "templates/support/agreement-hero.html" %}
{% include "templates/support/agreement-content.html" %}
{% include "templates/global/newsletter.html" %}
{% include "templates/global/github_stars.html" %}
{% endblock %}

View File

@ -0,0 +1,120 @@
<div id="contact" class="section pt-5">
<div class="container">
<p>This ClickHouse Subscription Agreement, including all referenced URLs, which are incorporated herein by reference (collectively, this “Agreement”), is entered into as of the date on which an applicable Order Form is fully executed (“Effective Date”), by and between the ClickHouse entity ("ClickHouse") set forth on such Order Form, and the entity identified thereon as the “Customer” (“Customer”).</p>
<p><strong>1. DEFINITIONS</strong><br /> Capitalized terms used herein have the meaning ascribed below, or where such terms are first used, as applicable.</p>
<p>1.1 <strong>"Affiliate"</strong> means, with respect to a party, any entity that controls, is controlled by, or which is under common control with, such party, where "control" means ownership of at least fifty percent (50%) of the outstanding voting shares of the entity, or the contractual right to establish policy for, and manage the operations of, the entity.</p>
<p>1.2 <strong>"Order Form"</strong> means an ordering document provided by ClickHouse pursuant to which Customer purchases Subscriptions under this Agreement.</p>
<p>1.3 <strong>"Qualifying PO"</strong> means a purchase order issued by customer for the purpose of purchasing a Subscription, which (i) references the number of an applicable Order Form provided to Customer by ClickHouse and (ii) clearly states the purchase order is subject to the terms and conditions of this Agreement.</p>
<p>1.4 <strong>"Software"</strong> means the ClickHouse software of the same name that is licensed for use under the Apache 2.0 license.</p>
<p>1.5 <strong>"Subscription"</strong> means Customer's right, for a fixed period of time, to receive Support Services, as set forth in the applicable Order Form.</p>
<p>1.6 <strong>"Subscription Term"</strong> means the period of time for which a Subscription is valid, as further described in Section 7.1 of this Agreement.</p>
<p>1.7 <strong>"Support Services"</strong> means maintenance and support services for the Software, as more fully described in the Support Services Policy.</p>
<p>1.8 "Support Services Policy" means ClickHouse's support services policy as further described at <a href="https://clickhouse.com/support/policy/">https://clickhouse.com/support/policy/</a>. ClickHouse reserves the right to reasonably modify the Support Services Policy during a Subscription Term, provided however, ClickHouse shall not materially diminish the level of Support Services during a Subscription Term. The effective date of each version of the Support Services Policy will be stated thereon, and ClickHouse agrees to archive copies of each version, and make the same available to Customer upon written request (e-mail sufficient). The parties agree that the Support Services Policy is hereby incorporated into these terms and conditions by this reference.</p>
<p><strong>2. AGREEMENT SCOPE AND PERFORMANCE OF SUPPORT SERVICES</strong></p>
<p>2.1 <u>Agreement Scope.</u> This Agreement includes terms and conditions applicable to Subscriptions for Support Services purchased under each Order Form entered into by the parties under Section 2.2 below, which Support Services may be used by Customer solely for Internal use and in connection with the use case(s) set forth on the applicable Order Form.</p>
<p>2.2 <u>Order for Support Services Subscriptions.</u> Orders for Subscriptions may be placed by Customer through (1) the execution of Order Forms with ClickHouse or (2) issuance by Customer of a Qualifying PO, which will be deemed to constitute, for the purposes of this Agreement, the execution by Customer of the referenced Order Form.</p>
<p>2.3 <u>Affiliates.</u> The parties agree that their respective Affiliates may also conduct business under this Agreement by entering into Order Forms, which in some cases may be subject to such additional and/or alternative terms and conditions to those contained in this Agreement as may be mutually agreed in the Order Form or an attachment thereto, as applicable. Accordingly, where Affiliates of the parties conduct business hereunder, references to Customer herein shall include any applicable Customer Affiliate, and references to ClickHouse herein shall include any applicable ClickHouse Affiliate. The parties agree that where either of them or one of their Affiliates enters into an Order Form with an Affiliate of the other party, that such Affiliate shall be solely responsible for performing all of its obligations under this Agreement in connection with such Order Form.</p>
<p>2.4 <u>Performance of Support Services.</u> Subject to Customers payment of all fees (as set forth in an applicable Order Form), ClickHouse will provide Customer with Support Services for the Software during an applicable Subscription Term in accordance with this Agreement and the Support Services Policy. Customer will reasonably cooperate with ClickHouse in connection with the Support Services, including, without limitation, by providing ClickHouse reasonable remote access to its installations, server cloud (or hosting provider), Software and equipment in connection therewith. Further, Customer will designate appropriately skilled personnel to serve as ClickHouses central contacts in connection with the use, operation and support of the Software. Customer understands that ClickHouses performance of Support Services is dependent in part on Customers cooperation, actions, and performance. ClickHouse shall not be responsible for any delays or interruptions in its performance of Support Services, or any claims arising therefrom, due to Customers lack of cooperation or acts or omissions. ClickHouse may use its Affiliates or subcontractors to provide Support Services to Customer, provided that ClickHouse remains responsible to Customer for performance.</p>
<p><strong>3. PAYMENT AND TAXES </strong></p>
<p>3.1 <u>Payment.</u> ClickHouse will invoice Customer for the fees due under each Order Form or otherwise under this Agreement, and Customer will pay such fees within thirty (30) days after receipt of an applicable invoice. All invoices will be paid in the currency set forth on the applicable Order Form. Payments will be made without right of set-off or chargeback. Except as otherwise expressly provided in this Agreement, any and all payments made by Customer pursuant to this Agreement or any Order Form are non-refundable, and all commitments to make any payments hereunder or under any Order Form are non-cancellable.</p>
<p>3.2 <u>Taxes.</u> All fees stated on an Order Form are exclusive of any applicable sales, use, value added and excise taxes levied upon the delivery or use of the taxable components, if any, of any Subscription purchased by Customer under this Agreement (collectively, “Taxes”). Taxes do not include any taxes on the net income of ClickHouse or any of its Affiliates. Unless Customer provides ClickHouse a valid state sales/use/excise tax exemption certificate or Direct Pay Permit, and provided that ClickHouse separately states any such taxes in the applicable invoice, Customer will pay and be solely responsible for all Taxes. If Customer is required by any foreign governmental authority to deduct or withhold any portion of the amount invoiced for the delivery or use of Support Services under this Agreement, Customer shall increase the sum paid to ClickHouse by an amount necessary for the total payment to ClickHouse equal to the amount originally invoiced.</p>
<p><strong>4. CONFIDENTIAL INFORMATION </strong></p>
<p>4.1 <u>Confidential Information.</u> Both parties acknowledge that, in the course of performing this Agreement, they may obtain information relating to products (such as goods, services, and software) of the other party, or relating to the parties themselves, which is of a confidential and proprietary nature ("Confidential Information"). Confidential Information includes materials and all communications concerning ClickHouse's or Customer's business and marketing strategies, including but not limited to employee and customer lists, customer profiles, project plans, design documents, product strategies and pricing data, research, advertising plans, leads and sources of supply, development activities, design and coding, interfaces with the Products, anything provided by either party to the other in connection with the Products and/or Support Services provided under this Agreement, including, without limitation, computer programs, technical drawings, algorithms, know-how, formulas, processes, ideas, inventions (whether patentable or not), schematics and other technical plans and other information of the parties which by its nature can be reasonably expected to be proprietary and confidential, whether it is presented in oral, printed, written, graphic or photographic or other tangible form (including information received, stored or transmitted electronically) even though specific designation as Confidential Information has not been made. Confidential Information also includes any notes, summaries, analyses of the foregoing that are prepared by the receiving party.</p>
<p>4.2 <u>Non-use and Non-disclosure.</u> The parties shall at all times, both during the Term and thereafter keep in trust and confidence all Confidential Information of the other party using commercially reasonable care (but in no event less than the same degree of care that the receiving party uses to protect its own Confidential Information) and shall not use such Confidential Information other than as necessary to carry out its duties under this Agreement, nor shall either party disclose any such Confidential Information to third parties other than to Affiliates or as necessary to carry out its duties under this Agreement without the other party's prior written consent, provided that each party shall be allowed to disclose Confidential Information of the other party to the extent that such disclosure is approved in writing by such other party, or necessary to enforce its rights under this Agreement.</p>
<p>4.3 <u>Non-Applicability.</u> The obligations of confidentiality shall not apply to information which (i) has entered the public domain or is otherwise publicly available, except where such entry or availability is the result of a party's breach of this Agreement; (ii) prior to disclosure hereunder was already in the receiving party's possession without restriction as evidenced by appropriate documentation; (iii) subsequent to disclosure hereunder is obtained by the receiving party on a non-confidential basis from a third party who has the right to disclose such information; or (iv) was developed by the receiving party without any use of any of the Confidential Information as evidenced by appropriate documentation.</p>
<p>4.4 <u>Terms of this Agreement.</u> Except as required by law or governmental regulation, neither party shall disclose, advertise, or publish the terms and conditions of this Agreement without the prior written consent of the other party, except that either party may disclose the terms of this Agreement to potential acquirers, referral partners involved in an applicable transaction, accountants, attorneys and Affiliates pursuant to the terms of a non-disclosure or confidentiality agreement. If Customer is using a third party provider to host a Product, then such provider may also receive, subject to a confidentiality obligation, information related to the terms of this Agreement or Customers usage of the applicable Product.</p>
<p>4.5 <u>Disclosure Required by Law.</u> Notwithstanding anything to the contrary herein, each party may disclose the other party's Confidential Information in order to comply with applicable law and/or an order from a court or other governmental body of competent jurisdiction, and, in connection with compliance with such an order only, if such party: (i) unless prohibited by law, gives the other party prior written notice to such disclosure if the time between that order and such disclosure reasonably permits or, if time does not permit, gives the other party written notice of such disclosure promptly after complying with that order and (ii) fully cooperates with the other party, at the other party's cost and expense, in seeking a protective order, or confidential treatment, or taking other measures to oppose or limit such disclosure. Each party must not release any more of the other party's Confidential Information than is, in the opinion of its counsel, reasonably necessary to comply with an applicable order.</p>
<p><strong>5. WARRANTIES AND DISCLAIMER OF WARRANTIES</strong></p>
<p>5.1 <u>Limited Support Services Performance Warranty.</u> ClickHouse warrants that it will perform the Support Services in a professional, workmanlike manner, consistent with generally accepted industry practice, and in accordance with the Support Services Policy. In the event of a breach of the foregoing warranty, ClickHouses sole obligation, and Customers exclusive remedy, shall be for ClickHouse to re-perform the applicable Support Services.</p>
<p>5.2 <u>Warranty Disclaimer.</u> EXCEPT AS SET FORTH IN SECTION 5.1 ABOVE, THE SUPPORT SERVICES ARE PROVIDED “AS IS” WITHOUT WARRANTY OF ANY KIND AND CLICKHOUSE MAKES NO ADDITIONAL WARRANTIES, WHETHER EXPRESSED, IMPLIED OR STATUTORY, REGARDING OR RELATING TO THE SUPPORT SERVICES OR ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT. TO THE MAXIMUM EXTENT PERMITTED UNDER APPLICABLE LAW, CLICKHOUSE SPECIFICALLY DISCLAIMS ALL IMPLIED WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NON-INFRINGEMENT WITH RESPECT TO THE SUPPORT SERVICES AND ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT. CUSTOMER UNDERSTANDS AND AGREES THAT THE SUPPORT SERVICES AND ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT ARE NOT DESIGNED OR INTENDED FOR USE IN THE OPERATION OF NUCLEAR FACILITIES, AIRCRAFT, WEAPONS SYSTEMS, OR LIFE SUPPORT SYSTEMS.</p>
<p><strong>6. LIMITATION OF LIABILITY</strong></p>
<p>6.1 <u>Excluded Damages.</u> IN NO EVENT SHALL CUSTOMER OR CLICKHOUSE, OR THEIR RESPECTIVE AFFILIATES, BE LIABLE FOR ANY LOSS OF PROFITS, LOSS OF USE, BUSINESS INTERRUPTION, LOSS OF DATA, COST OF SUBSTITUTE GOODS OR SERVICES, OR FOR ANY PUNITIVE, INDIRECT, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES OF ANY KIND IN CONNECTION WITH OR ARISING OUT OF THE PERFORMANCE OF OR FAILURE TO PERFORM THIS AGREEMENT, WHETHER ALLEGED AS A BREACH OF CONTRACT OR TORTIOUS CONDUCT, INCLUDING NEGLIGENCE, EVEN IF A PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.</p>
<p>6.2 <u>Damages Cap.</u> EXCEPT WITH RESPECT TO (I) A PARTYS BREACH OF ITS OBLIGATIONS UNDER SECTION 4, (II) AMOUNTS PAYABLE BY CUSTOMER UNDER SECTION 3 OF THIS AGREEMENT AND EACH ORDER FORM, AND (III) CUSTOMER'S VIOLATIONS OF THE USE RESTRICTIONS SET FORTH IN THIS AGREEMENT, IN NO EVENT SHALL CLICKHOUSE'S OR CUSTOMERS TOTAL, CUMULATIVE LIABILITY UNDER ANY ORDER FORM EXCEED THE AMOUNT PAID OR PAYABLE BY CUSTOMER TO CLICKHOUSE UNDER THIS AGREEMENT FOR THE AFFECTED SUPPORT SERVICES DELIVERED AND/OR MADE AVAILABLE TO CUSTOMER UNDER SUCH ORDER FORM FOR THE TWELVE (12) MONTH PERIOD IMMEDIATELY PRIOR TO THE FIRST EVENT GIVING RISE TO LIABILITY.</p>
<p>6.3 <u>Basis of the Bargain.</u> THE ALLOCATIONS OF LIABILITY IN THIS SECTION 6 REPRESENT THE AGREED AND BARGAINED FOR UNDERSTANDING OF THE PARTIES, AND THE COMPENSATION OF CLICKHOUSE FOR THE SUPPORT SERVICES PROVIDED HEREUNDER REFLECTS SUCH ALLOCATIONS. THE FOREGOING LIMITATIONS, EXCLUSIONS AND DISCLAIMERS WILL APPLY TO THE MAXIMUM EXTENT PERMITTED BY APPLICABLE LAW, EVEN IF ANY REMEDY FAILS IN ITS ESSENTIAL PURPOSE.</p>
<p><strong>7. TERM AND TERMINATION</strong></p>
<p>7.1 <u>Subscription Term.</u> The initial Subscription Term for each Subscription will commence and expire in accordance with the start date and end date set forth on the applicable Order Form, unless earlier terminated in in accordance with Section 7.3 below. Thereafter, each Subscription may be renewed for additional one (1) year periods upon the mutual written agreement of the parties. The initial Subscription Term, plus any subsequent renewal Subscription Term shall be the <strong>"Subscription Term"</strong>.</p>
<p>7.2 <u>Agreement Term.</u> This Agreement will commence on the Effective Date and, unless earlier terminated in accordance with Section 7.3(b) below, continue in force and effect for a period of two (2) years. Thereafter, the term of this Agreement shall automatically renew for additional one (1) year periods unless either party gives written notice to the other of its intention not to renew the Agreement at least thirty (30) days prior to the expiration of the then-current term. The initial term of this Agreement, plus any subsequent renewal term, shall be the <strong>"Term"</strong> of this Agreement. Notwithstanding any expiration of this Agreement, its terms will continue to apply to any Subscription that has not been terminated or for which the Subscription Term has not expired.</p>
<p>7.3 <u>Termination.</u></p>
<ol class="default" style="list-style-type: lower-alpha;">
<li><strong>Subscriptions.</strong> Each party may terminate a Subscription upon giving notice in writing to the other party if the non-terminating party commits a material breach of this Agreement with respect to such Subscription, and has failed to cure such breach within thirty (30) days following a request in writing from the notifying party to do so. Upon the termination or expiration of a Subscription, the rights and obligations of the parties with respect thereto will, subject to Section 7.4 below, cease, provided that termination of a Subscription under this subsection (a) will not result in termination of any other Subscriptions.</li>
<li><strong>Agreement.</strong> Either party may terminate this Agreement upon giving notice in writing to the other party if the non-terminating party commits a material breach of this Agreement with respect to any active Subscriptions hereunder, and has failed to cure such breach within thirty (30) days following a request in writing from the notifying party to do so. For the avoidance of doubt, termination of this Agreement under this subsection (b) will result in the termination of all Subscriptions and Order Forms.</li>
</ol>
<p>7.4 <u>Survival.</u> Upon the expiration or termination of an Order Form or this Agreement, (i) Customer shall have no further rights under any affected Subscription(s); and (ii) any payment obligations accrued under Section 3, as well as the provisions of Sections 1, 4, 5, 6, 7, 7.4 and 9 of this Agreement will survive such expiration or termination.</p>
<p><strong>8. GENERAL</strong></p>
<p>8.1 <u>Anti-Corruption.</u> Each party acknowledges that it is aware of, understands and has complied and will comply with, all applicable U.S. and foreign anti-corruption laws, including without limitation, the U.S. Foreign Corrupt Practices Act of 1977 and the U.K. Bribery Act of 2010, and similarly applicable anti-corruption and anti-bribery laws (<strong>"Anti-Corruption Laws"</strong>). Each party agrees that no one acting on its behalf will give, offer, agree or promise to give, or authorize the giving directly or indirectly, of any money or other thing of value, including travel, entertainment, or gifts, to anyone as an unlawful inducement or reward for favorable action or forbearance from action or the exercise of unlawful influence (a) to any governmental official or employee (including employees of government-owned and government-controlled corporations or agencies or public international organizations), (b) to any political party, official of a political party, or candidate, (c) to an intermediary for payment to any of the foregoing, or (d) to any other person or entity in a corrupt or improper effort to obtain or retain business or any commercial advantage, such as receiving a permit or license, or directing business to any person. Improper payments, provisions, bribes, kickbacks, influence payments, or other unlawful provisions to any person are prohibited under this Agreement.</p>
<p>8.2 <u>Assignment.</u> Neither party may assign this Agreement, in whole or in part, without the prior written consent of the other party, provided that no such consent will be required to assign this Agreement in its entirety to (i) an Affiliate that is able to satisfy the obligations of the assigning party under this Agreement or (ii) a successor in interest in connection with a merger, acquisition or sale of all or substantially all of the assigning party's assets. Any assignment in violation of this Section shall be void, ab initio, and of no effect. Subject to the foregoing, this Agreement is binding upon, inures to the benefit of and is enforceable by, the parties and their respective permitted successors and assigns.</p>
<p>8.3 <u>Attorneys' Fees.</u> If any action or proceeding, whether regulatory, administrative, at law or in equity is commenced or instituted to enforce or interpret any of the terms or provisions of this Agreement, the prevailing party in any such action or proceeding shall be entitled to recover its reasonable attorneys' fees, expert witness fees, costs of suit and expenses, in addition to any other relief to which such prevailing party may be entitled. As used herein, "prevailing party" includes without limitation, a party who dismisses an action for recovery hereunder in exchange for payment of the sums allegedly due, performance of covenants allegedly breached, or consideration substantially equal to the relief sought in the action.</p>
<p>8.4 <u>California Consumer Privacy Act (CCPA).</u> ClickHouse is a “Service Provider” as such term is defined under §1798.140(v) of the CCPA. As such ClickHouse shall not retain, use or disclose any personal information (as defined in the CCPA) received from Customer during the Term of this Agreement for any purpose other than the specific purpose of providing the Support Services specified in this Agreement or for such other business purpose as is specified in this Agreement.</p>
<p>8.5 Customer Identification. ClickHouse may identify Customer as a user of the Support Services, on its website, through a press release issued by ClickHouse and in other promotional materials.</p>
<p>8.6 <u>Feedback.</u> Customer, Customers Affiliates, and their respective agents, may volunteer feedback to ClickHouse, and/or its Affiliates, about the Support Services (<strong>“Feedback”</strong>). ClickHouse and its Affiliates shall be irrevocably entitled to use that Feedback, for any purpose and without any duty to account. provided that, in doing so, they may not breach their obligations of confidentiality under Section 4 of this Agreement. </p>
<p>8.7 <u>Force Majeure.</u> Except with respect to payment obligations, neither party will be liable for, or be considered to be in breach of, or in default under, this Agreement, as a result of any cause or condition beyond such party's reasonable control.</p>
<p>8.8 <u>Governing Law, Jurisdiction and Venue.</u></p>
<ol class="default" style="list-style-type: lower-alpha;">
<li><strong>Customers in California.</strong> If Customer is located in California (as determined by the Customer address on the applicable Order Form), this Agreement will be governed by the laws of the State of California, without regard to its conflict of laws principles, and all suits hereunder will be brought solely in Federal Court for the Northern District of California, or if that court lacks subject matter jurisdiction, in any California State Court located in Santa Clara County.</li>
<li><strong>Customers Outside of California.</strong> If Customer is located anywhere other than California (as determined by the Customer address on the applicable Order Form), this Agreement will be governed by the laws of the State of Delaware, without regard to its conflict of laws principles, and all suits hereunder will be brought solely in Federal Court for the District of Delaware, or if that court lacks subject matter jurisdiction, in any Delaware State Court located in Wilmington, Delaware.</li>
<li><strong>All Customers.</strong> This Agreement shall not be governed by the 1980 UN Convention on Contracts for the International Sale of Goods. The parties hereby irrevocably waive any and all claims and defenses either might otherwise have in any action or proceeding in any of the applicable courts set forth in (a) or (b) above, based upon any alleged lack of personal jurisdiction, improper venue, forum non conveniens, or any similar claim or defense.</li>
<li><strong>Equitable Relief.</strong> A breach or threatened breach, by either party of Section 4 may cause irreparable harm for which damages at law may not provide adequate relief, and therefore the non-breaching party shall be entitled to seek injunctive relief without being required to post a bond.</li>
</ol>
<p>8.9 <u>Non-waiver.</u> Any failure of either party to insist upon or enforce performance by the other party of any of the provisions of this Agreement or to exercise any rights or remedies under this Agreement will not be interpreted or construed as a waiver or relinquishment of such party's right to assert or rely upon such provision, right or remedy in that or any other instance.</p>
<p>8.10 <u>Notices.</u> Any notice or other communication under this Agreement given by either party to the other will be deemed to be properly given if given in writing and delivered in person or by e-mail, if acknowledged received by return e-mail or followed within one day by a delivered or mailed copy of such notice, or if mailed, properly addressed and stamped with the required postage, to the intended recipient at its address specified on an Order Form. Notices to ClickHouse may also be sent to <a href="mailto:legal@ClickHouse.com">legal@ClickHouse.com</a>. Either party may from time to time change its address for notices under this Section by giving the other party notice of the change in accordance with this Section.</p>
<p>8.11 <u>Relationship of the Parties.</u> The relationship of the parties hereunder shall be that of independent contractors, and nothing herein shall be deemed or construed to create any employment, agency or fiduciary relationship between the parties. Each party shall be solely responsible for the supervision, direction, control and payment of its personnel, including, without limitation, for taxes, deductions and withholdings, compensation and benefits, and nothing herein will be deemed to result in either party having an employer-employee relationship with the personnel of the other party.</p>
<p>8.12 <u>Severability.</u> If any provision of this Agreement is held to be invalid or unenforceable, the remaining portions will remain in full force and effect and such provision will be enforced to the maximum extent possible so as to give effect the intent of the parties and will be reformed to the extent necessary to make such provision valid and enforceable.</p>
<p>8.13 <u>Entire Agreement; Amendment.</u> This Agreement, together with any Order Forms executed by the parties, and the Support Services Policy, each of which is hereby incorporated herein by this reference, constitutes the entire agreement between the parties concerning the subject matter hereof, and it supersedes, and its terms govern, all prior proposals, agreements, or other communications between the parties, oral or written, regarding such subject matter. This Agreement may be executed in any number of counterparts, each of which when so executed and delivered shall be deemed an original, and all of which together shall constitute one and the same agreement. Execution of a scanned copy will have the same force and effect as execution of an original, and a scanned signature will be deemed an original and valid signature. In the event of any conflict between the terms and conditions of any of the foregoing documents, the conflict shall be resolved based on the following order of precedence: (i) an applicable Order Form (but only for the transaction thereunder), (ii) an applicable Addendum (including any exhibits, attachments and addenda thereto), (iii) this Agreement, and (iv) the Support Services Policy. For the avoidance of doubt, the parties hereby expressly acknowledge and agree that if Customer issues any purchase orders or similar documents in connection with its purchase of a Subscription, it shall do so only for the purpose of Section 2.2(2) or for its own internal, administrative purposes and not with the intent to provide any contractual terms. By entering into this Agreement, whether prior to or following receipt of Customer's purchase order or any similar document, the parties are hereby expressly showing their intention not to be contractually bound by the contents of any such purchase order or similar document, which are hereby deemed rejected and extraneous to this Agreement, and ClickHouse's performance of this Agreement shall not amount to: (i) an acceptance by conduct of any terms set out or referred to in the purchase order or similar document; (ii) an amendment of this Agreement, nor (iii) an agreement to amend this Agreement. This Agreement shall not be modified except by a subsequently dated, written amendment that expressly amends this Agreement and which is signed on behalf of ClickHouse and Customer by their duly authorized representatives. The parties agree that the terms and conditions of this Agreement are a result of mutual negotiations. Therefore, the rule of construction that any ambiguity shall apply against the drafter is not applicable and will not apply to this Agreement. Any ambiguity shall be reasonably construed as to its fair meaning and not strictly for or against one party regardless of who authored the ambiguous language.</p>
</div>
</div>

View File

@ -0,0 +1,10 @@
<div class="hero bg-primary-light d-flex align-items-center base-hero">
<div class="hero-bg other-hero"></div>
<div class="container pt-8 pt-lg-10 pt-xl-15 pb-8 pb-lg-10 pb-xl-15">
<h1 class="display-1 mb-0">
{{ _('Clickhouse, Inc.<br /> Subscription Agreement') }}
</h1>
</div>
</div>