mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into fix-redundant-linux-docs
This commit is contained in:
commit
00afead7ec
@ -11,10 +11,6 @@ DATASET="${TABLE}_v1.tar.xz"
|
||||
QUERIES_FILE="queries.sql"
|
||||
TRIES=3
|
||||
|
||||
AMD64_BIN_URL="https://builds.clickhouse.com/master/amd64/clickhouse"
|
||||
AARCH64_BIN_URL="https://builds.clickhouse.com/master/aarch64/clickhouse"
|
||||
POWERPC64_BIN_URL="https://builds.clickhouse.com/master/ppc64le/clickhouse"
|
||||
|
||||
# Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'.
|
||||
|
||||
FASTER_DOWNLOAD=wget
|
||||
@ -33,19 +29,59 @@ fi
|
||||
mkdir -p clickhouse-benchmark-$SCALE
|
||||
pushd clickhouse-benchmark-$SCALE
|
||||
|
||||
if [[ ! -f clickhouse ]]; then
|
||||
CPU=$(uname -m)
|
||||
if [[ ($CPU == x86_64) || ($CPU == amd64) ]]; then
|
||||
$FASTER_DOWNLOAD "$AMD64_BIN_URL"
|
||||
elif [[ $CPU == aarch64 ]]; then
|
||||
$FASTER_DOWNLOAD "$AARCH64_BIN_URL"
|
||||
elif [[ $CPU == powerpc64le ]]; then
|
||||
$FASTER_DOWNLOAD "$POWERPC64_BIN_URL"
|
||||
else
|
||||
echo "Unsupported CPU type: $CPU"
|
||||
OS=$(uname -s)
|
||||
ARCH=$(uname -m)
|
||||
|
||||
DIR=
|
||||
|
||||
if [ "${OS}" = "Linux" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="amd64"
|
||||
elif [ "${ARCH}" = "aarch64" ]
|
||||
then
|
||||
DIR="aarch64"
|
||||
elif [ "${ARCH}" = "powerpc64le" ]
|
||||
then
|
||||
DIR="powerpc64le"
|
||||
fi
|
||||
elif [ "${OS}" = "FreeBSD" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="freebsd"
|
||||
elif [ "${ARCH}" = "aarch64" ]
|
||||
then
|
||||
DIR="freebsd-aarch64"
|
||||
elif [ "${ARCH}" = "powerpc64le" ]
|
||||
then
|
||||
DIR="freebsd-powerpc64le"
|
||||
fi
|
||||
elif [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="macos"
|
||||
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
|
||||
then
|
||||
DIR="macos-aarch64"
|
||||
fi
|
||||
fi
|
||||
|
||||
if [ -z "${DIR}" ]
|
||||
then
|
||||
echo "The '${OS}' operating system with the '${ARCH}' architecture is not supported."
|
||||
exit 1
|
||||
fi
|
||||
fi
|
||||
|
||||
URL="https://builds.clickhouse.com/master/${DIR}/clickhouse"
|
||||
echo
|
||||
echo "Will download ${URL}"
|
||||
echo
|
||||
curl -O "${URL}" && chmod a+x clickhouse || exit 1
|
||||
echo
|
||||
echo "Successfully downloaded the ClickHouse binary"
|
||||
|
||||
chmod a+x clickhouse
|
||||
|
||||
@ -88,7 +124,12 @@ echo
|
||||
|
||||
cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do
|
||||
sync
|
||||
if [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
sudo purge > /dev/null
|
||||
else
|
||||
echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null
|
||||
fi
|
||||
|
||||
echo -n "["
|
||||
for i in $(seq 1 $TRIES); do
|
||||
@ -104,6 +145,24 @@ echo
|
||||
echo "Benchmark complete. System info:"
|
||||
echo
|
||||
|
||||
if [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
echo '----Version, build id-----------'
|
||||
./clickhouse local --query "SELECT format('Version: {}', version())"
|
||||
sw_vers | grep BuildVersion
|
||||
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
|
||||
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
|
||||
echo '----CPU-------------------------'
|
||||
sysctl hw.model
|
||||
sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize'
|
||||
echo '----Disk Free and Total--------'
|
||||
df -h .
|
||||
echo '----Memory Free and Total-------'
|
||||
vm_stat
|
||||
echo '----Physical Memory Amount------'
|
||||
ls -l /var/vm
|
||||
echo '--------------------------------'
|
||||
else
|
||||
echo '----Version, build id-----------'
|
||||
./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())"
|
||||
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
|
||||
@ -126,5 +185,5 @@ cat /proc/mdstat
|
||||
#echo '----All Hardware Info-----------'
|
||||
#lshw
|
||||
echo '--------------------------------'
|
||||
|
||||
fi
|
||||
echo
|
||||
|
@ -38,6 +38,18 @@ Alternatively you can perform benchmark in the following steps.
|
||||
wget https://builds.clickhouse.com/master/amd64/clickhouse
|
||||
# For aarch64:
|
||||
wget https://builds.clickhouse.com/master/aarch64/clickhouse
|
||||
# For powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
|
||||
# For freebsd:
|
||||
wget https://builds.clickhouse.com/master/freebsd/clickhouse
|
||||
# For freebsd-aarch64:
|
||||
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
|
||||
# For freebsd-powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
|
||||
# For macos:
|
||||
wget https://builds.clickhouse.com/master/macos/clickhouse
|
||||
# For macos-aarch64:
|
||||
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
|
||||
# Then do:
|
||||
chmod a+x clickhouse
|
||||
```
|
||||
|
@ -20,9 +20,21 @@ toc_title: "\u30CF\u30FC\u30C9\u30A6\u30A7\u30A2\u8A66\u9A13"
|
||||
<!-- -->
|
||||
|
||||
# For amd64:
|
||||
wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578163263_binary/clickhouse
|
||||
wget https://builds.clickhouse.com/master/amd64/clickhouse
|
||||
# For aarch64:
|
||||
wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578161264_binary/clickhouse
|
||||
wget https://builds.clickhouse.com/master/aarch64/clickhouse
|
||||
# For powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
|
||||
# For freebsd:
|
||||
wget https://builds.clickhouse.com/master/freebsd/clickhouse
|
||||
# For freebsd-aarch64:
|
||||
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
|
||||
# For freebsd-powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
|
||||
# For macos:
|
||||
wget https://builds.clickhouse.com/master/macos/clickhouse
|
||||
# For macos-aarch64:
|
||||
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
|
||||
# Then do:
|
||||
chmod a+x clickhouse
|
||||
|
||||
|
@ -1 +0,0 @@
|
||||
../../../en/faq/integration/file-export.md
|
37
docs/zh/faq/integration/file-export.md
Normal file
37
docs/zh/faq/integration/file-export.md
Normal file
@ -0,0 +1,37 @@
|
||||
---
|
||||
title: 如何从 ClickHouse 导出数据到一个文件?
|
||||
toc_hidden: true
|
||||
toc_priority: 10
|
||||
---
|
||||
|
||||
# 如何从 ClickHouse 导出数据到一个文件? {#how-to-export-to-file}
|
||||
|
||||
## 使用 INTO OUTFILE 语法 {#using-into-outfile-clause}
|
||||
|
||||
加一个 [INTO OUTFILE](../../sql-reference/statements/select/into-outfile.md#into-outfile-clause) 语法到你的查询语句中.
|
||||
|
||||
例如:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM table INTO OUTFILE 'file'
|
||||
```
|
||||
|
||||
ClickHouse 默认使用[TabSeparated](../../interfaces/formats.md#tabseparated) 格式写入数据. 修改[数据格式](../../interfaces/formats.md), 请用 [FORMAT 语法](../../sql-reference/statements/select/format.md#format-clause).
|
||||
|
||||
例如:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM table INTO OUTFILE 'file' FORMAT CSV
|
||||
```
|
||||
|
||||
## 使用一个文件引擎表 {#using-a-file-engine-table}
|
||||
|
||||
查看 [File](../../engines/table-engines/special/file.md) 表引擎.
|
||||
|
||||
## 使用命令行重定向 {#using-command-line-redirection}
|
||||
|
||||
``` bash
|
||||
$ clickhouse-client --query "SELECT * from table" --format FormatName > result.txt
|
||||
```
|
||||
|
||||
查看 [clickhouse-client](../../interfaces/cli.md).
|
@ -36,6 +36,18 @@ chmod a+x ./hardware.sh
|
||||
wget https://builds.clickhouse.com/master/amd64/clickhouse
|
||||
# For aarch64:
|
||||
wget https://builds.clickhouse.com/master/aarch64/clickhouse
|
||||
# For powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
|
||||
# For freebsd:
|
||||
wget https://builds.clickhouse.com/master/freebsd/clickhouse
|
||||
# For freebsd-aarch64:
|
||||
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
|
||||
# For freebsd-powerpc64le:
|
||||
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
|
||||
# For macos:
|
||||
wget https://builds.clickhouse.com/master/macos/clickhouse
|
||||
# For macos-aarch64:
|
||||
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
|
||||
# Then do:
|
||||
chmod a+x clickhouse
|
||||
```
|
||||
|
@ -1 +0,0 @@
|
||||
../../../../en/sql-reference/statements/alter/role.md
|
16
docs/zh/sql-reference/statements/alter/role.md
Normal file
16
docs/zh/sql-reference/statements/alter/role.md
Normal file
@ -0,0 +1,16 @@
|
||||
---
|
||||
toc_priority: 46
|
||||
toc_title: 角色
|
||||
---
|
||||
|
||||
## 操作角色 {#alter-role-statement}
|
||||
|
||||
修改角色.
|
||||
|
||||
语法示例:
|
||||
|
||||
``` sql
|
||||
ALTER ROLE [IF EXISTS] name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1]
|
||||
[, name2 [ON CLUSTER cluster_name2] [RENAME TO new_name2] ...]
|
||||
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
|
||||
```
|
@ -229,11 +229,11 @@ public:
|
||||
static bool cancelled() { return exit_on_signal.test(); }
|
||||
};
|
||||
|
||||
/// This signal handler is set only for sigint.
|
||||
/// This signal handler is set only for SIGINT.
|
||||
void interruptSignalHandler(int signum)
|
||||
{
|
||||
if (exit_on_signal.test_and_set())
|
||||
_exit(signum);
|
||||
_exit(128 + signum);
|
||||
}
|
||||
|
||||
|
||||
@ -702,7 +702,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
|
||||
/// Also checks if query execution should be cancelled.
|
||||
void ClientBase::receiveResult(ASTPtr parsed_query)
|
||||
{
|
||||
bool cancelled = false;
|
||||
QueryInterruptHandler query_interrupt_handler;
|
||||
|
||||
// TODO: get the poll_interval from commandline.
|
||||
@ -773,7 +772,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query)
|
||||
/// Receive a part of the result, or progress info or an exception and process it.
|
||||
/// Returns true if one should continue receiving packets.
|
||||
/// Output of result is suppressed if query was cancelled.
|
||||
bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
|
||||
bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_)
|
||||
{
|
||||
Packet packet = connection->receivePacket();
|
||||
|
||||
@ -783,7 +782,7 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
|
||||
return true;
|
||||
|
||||
case Protocol::Server::Data:
|
||||
if (!cancelled)
|
||||
if (!cancelled_)
|
||||
onData(packet.block, parsed_query);
|
||||
return true;
|
||||
|
||||
@ -796,12 +795,12 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
|
||||
return true;
|
||||
|
||||
case Protocol::Server::Totals:
|
||||
if (!cancelled)
|
||||
if (!cancelled_)
|
||||
onTotals(packet.block, parsed_query);
|
||||
return true;
|
||||
|
||||
case Protocol::Server::Extremes:
|
||||
if (!cancelled)
|
||||
if (!cancelled_)
|
||||
onExtremes(packet.block, parsed_query);
|
||||
return true;
|
||||
|
||||
@ -1265,6 +1264,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
|
||||
{
|
||||
resetOutput();
|
||||
have_error = false;
|
||||
cancelled = false;
|
||||
client_exception.reset();
|
||||
server_exception.reset();
|
||||
|
||||
@ -1392,6 +1392,9 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
|
||||
String & query_to_execute, ASTPtr & parsed_query, const String & all_queries_text,
|
||||
std::optional<Exception> & current_exception)
|
||||
{
|
||||
if (!is_interactive && cancelled)
|
||||
return MultiQueryProcessingStage::QUERIES_END;
|
||||
|
||||
if (this_query_begin >= all_queries_end)
|
||||
return MultiQueryProcessingStage::QUERIES_END;
|
||||
|
||||
|
@ -108,7 +108,7 @@ protected:
|
||||
|
||||
private:
|
||||
void receiveResult(ASTPtr parsed_query);
|
||||
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled);
|
||||
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_);
|
||||
void receiveLogs(ASTPtr parsed_query);
|
||||
bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query);
|
||||
bool receiveEndOfQuery();
|
||||
@ -259,6 +259,8 @@ protected:
|
||||
};
|
||||
|
||||
std::vector<HostAndPort> hosts_and_ports{};
|
||||
|
||||
bool cancelled = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -184,6 +184,9 @@ void AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
|
||||
if (!FormatFactory::instance().isInputFormat(insert_query.format))
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown input format {}", insert_query.format);
|
||||
|
||||
/// For table functions we check access while executing
|
||||
/// InterpreterInsertQuery::getTable() -> ITableFunction::execute().
|
||||
if (insert_query.table_id)
|
||||
query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames());
|
||||
|
||||
String bytes;
|
||||
@ -411,7 +414,7 @@ try
|
||||
};
|
||||
|
||||
std::shared_ptr<ISimpleTransform> adding_defaults_transform;
|
||||
if (insert_context->getSettingsRef().input_format_defaults_for_omitted_fields)
|
||||
if (insert_context->getSettingsRef().input_format_defaults_for_omitted_fields && insert_query.table_id)
|
||||
{
|
||||
StoragePtr storage = DatabaseCatalog::instance().getTable(insert_query.table_id, insert_context);
|
||||
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
|
@ -642,6 +642,11 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
properties.indices = as_storage_metadata->getSecondaryIndices();
|
||||
properties.projections = as_storage_metadata->getProjections().clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Only MergeTree support TTL
|
||||
properties.columns.resetColumnTTLs();
|
||||
}
|
||||
|
||||
properties.constraints = as_storage_metadata->getConstraints();
|
||||
}
|
||||
|
@ -60,6 +60,18 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query)
|
||||
{
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
TableFunctionPtr table_function_ptr = factory.get(query.table_function, getContext());
|
||||
|
||||
/// If table function needs structure hint from select query
|
||||
/// we can create a temporary pipeline and get the header.
|
||||
if (query.select && table_function_ptr->needStructureHint())
|
||||
{
|
||||
InterpreterSelectWithUnionQuery interpreter_select{
|
||||
query.select, getContext(), SelectQueryOptions(QueryProcessingStage::Complete, 1)};
|
||||
QueryPipelineBuilder tmp_pipeline = interpreter_select.buildQueryPipeline();
|
||||
ColumnsDescription structure_hint{tmp_pipeline.getHeader().getNamesAndTypesList()};
|
||||
table_function_ptr->setStructureHint(structure_hint);
|
||||
}
|
||||
|
||||
return table_function_ptr->execute(query.table_function, getContext(), table_function_ptr->getName());
|
||||
}
|
||||
|
||||
@ -283,6 +295,9 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
|
||||
auto query_sample_block = getSampleBlock(query, table, metadata_snapshot);
|
||||
|
||||
/// For table functions we check access while executing
|
||||
/// getTable() -> ITableFunction::execute().
|
||||
if (!query.table_function)
|
||||
getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames());
|
||||
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -112,9 +113,7 @@ std::shared_ptr<TSystemLog> createSystemLog(
|
||||
}
|
||||
|
||||
|
||||
/// returns CREATE TABLE query, but with removed:
|
||||
/// - UUID
|
||||
/// - SETTINGS (for MergeTree)
|
||||
/// returns CREATE TABLE query, but with removed UUID
|
||||
/// That way it can be used to compare with the SystemLog::getCreateTableQuery()
|
||||
ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context)
|
||||
{
|
||||
@ -123,11 +122,6 @@ ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context)
|
||||
auto & old_create_query_ast = old_ast->as<ASTCreateQuery &>();
|
||||
/// Reset UUID
|
||||
old_create_query_ast.uuid = UUIDHelpers::Nil;
|
||||
/// Existing table has default settings (i.e. `index_granularity = 8192`), reset them.
|
||||
if (ASTStorage * storage = old_create_query_ast.storage)
|
||||
{
|
||||
storage->reset(storage->settings);
|
||||
}
|
||||
return old_ast;
|
||||
}
|
||||
|
||||
@ -476,6 +470,11 @@ ASTPtr SystemLog<LogElement>::getCreateTableQuery()
|
||||
"Storage to create table for " + LogElement::name(), 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
create->set(create->storage, storage_ast);
|
||||
|
||||
/// Write additional (default) settings for MergeTree engine to make it make it possible to compare ASTs
|
||||
/// and recreate tables on settings changes.
|
||||
auto storage_settings = std::make_unique<MergeTreeSettings>(getContext()->getMergeTreeSettings());
|
||||
storage_settings->loadFromQuery(*create->storage);
|
||||
|
||||
return create;
|
||||
}
|
||||
|
||||
|
@ -37,7 +37,7 @@ int CertificateReloader::setCertificate(SSL * ssl)
|
||||
return -1;
|
||||
|
||||
SSL_use_certificate(ssl, const_cast<X509 *>(current->cert.certificate()));
|
||||
SSL_use_RSAPrivateKey(ssl, current->key.impl()->getRSA());
|
||||
SSL_use_PrivateKey(ssl, const_cast<EVP_PKEY *>(static_cast<const EVP_PKEY *>(current->key)));
|
||||
|
||||
int err = SSL_check_private_key(ssl);
|
||||
if (err != 1)
|
||||
|
@ -74,7 +74,7 @@ private:
|
||||
struct Data
|
||||
{
|
||||
Poco::Crypto::X509Certificate cert;
|
||||
Poco::Crypto::RSAKey key;
|
||||
Poco::Crypto::EVPPKey key;
|
||||
|
||||
Data(std::string cert_path, std::string key_path);
|
||||
};
|
||||
|
@ -635,6 +635,22 @@ ColumnsDescription::ColumnTTLs ColumnsDescription::getColumnTTLs() const
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ColumnsDescription::resetColumnTTLs()
|
||||
{
|
||||
std::vector<ColumnDescription> old_columns;
|
||||
old_columns.reserve(columns.size());
|
||||
for (const auto & col : columns)
|
||||
old_columns.emplace_back(col);
|
||||
|
||||
columns.clear();
|
||||
|
||||
for (auto & col : old_columns)
|
||||
{
|
||||
col.ttl.reset();
|
||||
add(col);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
String ColumnsDescription::toString() const
|
||||
{
|
||||
|
@ -104,6 +104,7 @@ public:
|
||||
|
||||
using ColumnTTLs = std::unordered_map<String, ASTPtr>;
|
||||
ColumnTTLs getColumnTTLs() const;
|
||||
void resetColumnTTLs();
|
||||
|
||||
bool has(const String & column_name) const;
|
||||
bool hasNested(const String & column_name) const;
|
||||
|
@ -8,8 +8,10 @@
|
||||
#include <arrow/api.h>
|
||||
#include <arrow/io/api.h>
|
||||
#include <arrow/status.h>
|
||||
#include <arrow/filesystem/filesystem.h>
|
||||
#include <orc/OrcFile.hh>
|
||||
#include <orc/Reader.hh>
|
||||
#include <orc/Statistics.hh>
|
||||
#include <parquet/arrow/reader.h>
|
||||
#include <parquet/file_reader.h>
|
||||
#include <parquet/statistics.h>
|
||||
|
@ -7,8 +7,6 @@
|
||||
#include <memory>
|
||||
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <orc/Statistics.hh>
|
||||
#include <arrow/filesystem/filesystem.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Block.h>
|
||||
@ -18,6 +16,8 @@
|
||||
namespace orc
|
||||
{
|
||||
class Reader;
|
||||
class Statistics;
|
||||
class ColumnStatistics;
|
||||
}
|
||||
|
||||
namespace parquet
|
||||
@ -36,6 +36,11 @@ namespace io
|
||||
class RandomAccessFile;
|
||||
}
|
||||
|
||||
namespace fs
|
||||
{
|
||||
class FileSystem;
|
||||
}
|
||||
|
||||
class Buffer;
|
||||
}
|
||||
|
||||
|
@ -12,9 +12,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageFileBlockInputStream;
|
||||
class StorageFileBlockOutputStream;
|
||||
|
||||
class StorageFile final : public shared_ptr_helper<StorageFile>, public IStorage
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageFile>;
|
||||
|
@ -1,9 +1,17 @@
|
||||
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
add_headers_and_sources(clickhouse_table_functions .)
|
||||
if (TARGET ch_contrib::hivemetastore)
|
||||
add_headers_and_sources(clickhouse_table_functions Hive)
|
||||
endif ()
|
||||
|
||||
list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp TableFunctionFactory.cpp)
|
||||
list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h)
|
||||
|
||||
add_library(clickhouse_table_functions ${clickhouse_table_functions_sources})
|
||||
if (TARGET ch_contrib::hivemetastore)
|
||||
target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms ch_contrib::hivemetastore ch_contrib::hdfs)
|
||||
else ()
|
||||
target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms)
|
||||
endif ()
|
||||
|
||||
|
||||
|
91
src/TableFunctions/Hive/TableFunctionHive.cpp
Normal file
91
src/TableFunctions/Hive/TableFunctionHive.cpp
Normal file
@ -0,0 +1,91 @@
|
||||
#include <TableFunctions/Hive/TableFunctionHive.h>
|
||||
#if USE_HIVE
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ParserPartition.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Storages/Hive/HiveSettings.h>
|
||||
#include <Storages/Hive/StorageHive.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
void TableFunctionHive::parseArguments(const ASTPtr & ast_function_, ContextPtr context_)
|
||||
{
|
||||
ASTs & args_func = ast_function_->children;
|
||||
if (args_func.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName());
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
const auto message = fmt::format(
|
||||
"The signature of function {} is:\n"
|
||||
" - hive_url, hive_database, hive_table, structure, partition_by_keys",
|
||||
getName());
|
||||
|
||||
if (args.size() != 5)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, message);
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
||||
|
||||
hive_metastore_url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
hive_database = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
hive_table = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
table_structure = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
partition_by_def = args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
actual_columns = parseColumnsListFromString(table_structure, context_);
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionHive::getActualTableStructure(ContextPtr /*context_*/) const { return actual_columns; }
|
||||
|
||||
StoragePtr TableFunctionHive::executeImpl(
|
||||
const ASTPtr & /*ast_function_*/,
|
||||
ContextPtr context_,
|
||||
const std::string & table_name_,
|
||||
ColumnsDescription /*cached_columns_*/) const
|
||||
{
|
||||
const Settings & settings = context_->getSettings();
|
||||
ParserLambdaExpression partition_by_parser;
|
||||
ASTPtr partition_by_ast = parseQuery(
|
||||
partition_by_parser,
|
||||
"(" + partition_by_def + ")",
|
||||
"partition by declaration list",
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth);
|
||||
StoragePtr storage;
|
||||
storage = StorageHive::create(
|
||||
hive_metastore_url,
|
||||
hive_database,
|
||||
hive_table,
|
||||
StorageID(getDatabaseName(), table_name_),
|
||||
actual_columns,
|
||||
ConstraintsDescription{},
|
||||
"",
|
||||
partition_by_ast,
|
||||
std::make_unique<HiveSettings>(),
|
||||
context_);
|
||||
|
||||
return storage;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionHive(TableFunctionFactory & factory_) { factory_.registerFunction<TableFunctionHive>(); }
|
||||
|
||||
}
|
||||
#endif
|
38
src/TableFunctions/Hive/TableFunctionHive.h
Normal file
38
src/TableFunctions/Hive/TableFunctionHive.h
Normal file
@ -0,0 +1,38 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
#if USE_HIVE
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Poco/Logger.h>
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class TableFunctionHive : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hive";
|
||||
static constexpr auto storage_type_name = "hive";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return storage_type_name; }
|
||||
ColumnsDescription getActualTableStructure(ContextPtr) const override;
|
||||
void parseArguments(const ASTPtr & ast_function_, ContextPtr context_) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * logger = &Poco::Logger::get("TableFunctionHive");
|
||||
|
||||
String cluster_name;
|
||||
String hive_metastore_url;
|
||||
String hive_database;
|
||||
String hive_table;
|
||||
String table_structure;
|
||||
String partition_by_def;
|
||||
|
||||
ColumnsDescription actual_columns;
|
||||
};
|
||||
}
|
||||
#endif
|
@ -52,6 +52,16 @@ public:
|
||||
/// Returns actual table structure probably requested from remote server, may fail
|
||||
virtual ColumnsDescription getActualTableStructure(ContextPtr /*context*/) const = 0;
|
||||
|
||||
/// Check if table function needs a structure hint from SELECT query in case of
|
||||
/// INSERT INTO FUNCTION ... SELECT ...
|
||||
/// It's used for schema inference.
|
||||
virtual bool needStructureHint() const { return false; }
|
||||
|
||||
/// Set a structure hint from SELECT query in case of
|
||||
/// INSERT INTO FUNCTION ... SELECT ...
|
||||
/// This hint could be used not to repeat schema in function arguments.
|
||||
virtual void setStructureHint(const ColumnsDescription &) {}
|
||||
|
||||
/// Create storage according to the query.
|
||||
StoragePtr
|
||||
execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}, bool use_global_context = false) const;
|
||||
|
@ -95,6 +95,9 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ColumnsDescription columns;
|
||||
if (structure != "auto")
|
||||
columns = parseColumnsListFromString(structure, context);
|
||||
else if (!structure_hint.empty())
|
||||
columns = structure_hint;
|
||||
|
||||
StoragePtr storage = getStorage(filename, format, columns, context, table_name, compression_method);
|
||||
storage->startup();
|
||||
return storage;
|
||||
|
@ -12,6 +12,10 @@ class Context;
|
||||
*/
|
||||
class ITableFunctionFileLike : public ITableFunction
|
||||
{
|
||||
public:
|
||||
bool needStructureHint() const override { return structure == "auto"; }
|
||||
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
protected:
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
@ -20,6 +24,7 @@ protected:
|
||||
String format = "auto";
|
||||
String structure = "auto";
|
||||
String compression_method = "auto";
|
||||
ColumnsDescription structure_hint;
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||
|
@ -41,6 +41,7 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context
|
||||
return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context);
|
||||
}
|
||||
|
||||
|
||||
return parseColumnsListFromString(structure, context);
|
||||
}
|
||||
|
||||
|
@ -168,6 +168,8 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context
|
||||
ColumnsDescription columns;
|
||||
if (s3_configuration->structure != "auto")
|
||||
columns = parseColumnsListFromString(s3_configuration->structure, context);
|
||||
else if (!structure_hint.empty())
|
||||
columns = structure_hint;
|
||||
|
||||
StoragePtr storage = StorageS3::create(
|
||||
s3_uri,
|
||||
|
@ -25,6 +25,10 @@ public:
|
||||
}
|
||||
bool hasStaticStructure() const override { return s3_configuration->structure != "auto"; }
|
||||
|
||||
bool needStructureHint() const override { return s3_configuration->structure == "auto"; }
|
||||
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
protected:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function,
|
||||
@ -38,6 +42,7 @@ protected:
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
std::optional<StorageS3Configuration> s3_configuration;
|
||||
ColumnsDescription structure_hint;
|
||||
};
|
||||
|
||||
class TableFunctionCOS : public TableFunctionS3
|
||||
|
@ -31,6 +31,10 @@ void registerTableFunctions()
|
||||
registerTableFunctionHDFSCluster(factory);
|
||||
#endif
|
||||
|
||||
#if USE_HIVE
|
||||
registerTableFunctionHive(factory);
|
||||
#endif
|
||||
|
||||
registerTableFunctionODBC(factory);
|
||||
registerTableFunctionJDBC(factory);
|
||||
|
||||
|
@ -29,6 +29,10 @@ void registerTableFunctionHDFS(TableFunctionFactory & factory);
|
||||
void registerTableFunctionHDFSCluster(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_HIVE
|
||||
void registerTableFunctionHive(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
||||
void registerTableFunctionODBC(TableFunctionFactory & factory);
|
||||
void registerTableFunctionJDBC(TableFunctionFactory & factory);
|
||||
|
||||
|
@ -107,6 +107,20 @@ def test_parquet_groupby_with_cache(started_cluster):
|
||||
2021-11-16 2
|
||||
"""
|
||||
assert result == expected_result
|
||||
|
||||
def test_parquet_groupby_by_hive_function(started_cluster):
|
||||
logging.info('Start testing groupby ...')
|
||||
node = started_cluster.instances['h0_0_0']
|
||||
result = node.query("""
|
||||
SELECT day, count(*) FROM hive('thrift://hivetest:9083', 'test', 'demo', '`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)', 'day') group by day order by day
|
||||
""")
|
||||
expected_result = """2021-11-01 1
|
||||
2021-11-05 2
|
||||
2021-11-11 1
|
||||
2021-11-16 2
|
||||
"""
|
||||
assert result == expected_result
|
||||
|
||||
def test_cache_read_bytes(started_cluster):
|
||||
node = started_cluster.instances['h0_0_0']
|
||||
node.query("set input_format_parquet_allow_missing_columns = true")
|
||||
|
16
tests/integration/test_reload_certificate/configs/ECcert.crt
Normal file
16
tests/integration/test_reload_certificate/configs/ECcert.crt
Normal file
@ -0,0 +1,16 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIICkzCCAhigAwIBAgIUcrahhUuSDdw60Wyfo2E4kVUWWQ8wCgYIKoZIzj0EAwIw
|
||||
fzELMAkGA1UEBhMCQ04xEzARBgNVBAgMClNvbWUtU3RhdGUxDTALBgNVBAcMBGNp
|
||||
dHkxEDAOBgNVBAoMB2NvbXBhbnkxEDAOBgNVBAsMB3NlY3Rpb24xEjAQBgNVBAMM
|
||||
CWxvY2FsaG9zdDEUMBIGCSqGSIb3DQEJARYFZW1haWwwIBcNMjIwMjI3MTg1NzQz
|
||||
WhgPMjEyMjAyMDMxODU3NDNaMH8xCzAJBgNVBAYTAkNOMRMwEQYDVQQIDApTb21l
|
||||
LVN0YXRlMQ0wCwYDVQQHDARjaXR5MRAwDgYDVQQKDAdjb21wYW55MRAwDgYDVQQL
|
||||
DAdzZWN0aW9uMRIwEAYDVQQDDAlsb2NhbGhvc3QxFDASBgkqhkiG9w0BCQEWBWVt
|
||||
YWlsMHYwEAYHKoZIzj0CAQYFK4EEACIDYgAEgoPY/r89/83zHzmpbsA+kW3YflVQ
|
||||
tKXO8Kl7ki5q+v1qUu3xmr4HttNxvHLOCfK798KMGg9y+NO5y4D4D2ZgLGxkNt8X
|
||||
yWvhkbe3xKdGSqBpplbLT+M9FtmQ6tzzzFJVo1MwUTAdBgNVHQ4EFgQUmpLPeJBD
|
||||
ID5s1AeWsVIEt6Z/ca0wHwYDVR0jBBgwFoAUmpLPeJBDID5s1AeWsVIEt6Z/ca0w
|
||||
DwYDVR0TAQH/BAUwAwEB/zAKBggqhkjOPQQDAgNpADBmAjEAv4uNU4NgprBgNQxk
|
||||
fIZpJCf/TpisuVsLUHXl8JrMVKKVUf7zr59GH2yiOoukfD5hAjEAlCohSA6/Ken4
|
||||
JWkKPCrfnsBZ7VX8Y+4ZqLKuG+IGAu2vQTg+Jc6M23M1vEgi1dqf
|
||||
-----END CERTIFICATE-----
|
@ -0,0 +1,6 @@
|
||||
-----BEGIN PRIVATE KEY-----
|
||||
MIG2AgEAMBAGByqGSM49AgEGBSuBBAAiBIGeMIGbAgEBBDAJbfB78wfRHn5A4x3e
|
||||
EAqrFk/hbBD+c8snbFgjQqxg4qTcp154Rc01B9V0US27MJuhZANiAASCg9j+vz3/
|
||||
zfMfOaluwD6Rbdh+VVC0pc7wqXuSLmr6/WpS7fGavge203G8cs4J8rv3wowaD3L4
|
||||
07nLgPgPZmAsbGQ23xfJa+GRt7fEp0ZKoGmmVstP4z0W2ZDq3PPMUlU=
|
||||
-----END PRIVATE KEY-----
|
@ -6,6 +6,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance('node', main_configs=["configs/first.crt", "configs/first.key",
|
||||
"configs/second.crt", "configs/second.key",
|
||||
"configs/ECcert.crt", "configs/ECcert.key",
|
||||
"configs/cert.xml"])
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
@ -73,3 +74,27 @@ def test_first_than_second_cert():
|
||||
assert False
|
||||
except:
|
||||
assert True
|
||||
|
||||
def test_ECcert_reload():
|
||||
# Set first key
|
||||
change_config_to_key('first')
|
||||
|
||||
# Command with correct certificate
|
||||
assert node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='first'),
|
||||
'https://localhost:8443/']) == 'Ok.\n'
|
||||
|
||||
# Change to other key
|
||||
change_config_to_key('ECcert')
|
||||
|
||||
# Command with correct certificate
|
||||
assert node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='ECcert'),
|
||||
'https://localhost:8443/']) == 'Ok.\n'
|
||||
|
||||
# Command with wrong certificate
|
||||
# Same as previous
|
||||
try:
|
||||
node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='first'),
|
||||
'https://localhost:8443/'])
|
||||
assert False
|
||||
except:
|
||||
assert True
|
||||
|
@ -421,6 +421,16 @@ def test_schema_inference_with_globs(started_cluster):
|
||||
assert(sorted(result.split()) == ['0', '\\N'])
|
||||
|
||||
|
||||
def test_insert_select_schema_inference(started_cluster):
|
||||
node1.query(f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x")
|
||||
|
||||
result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/test.native.zst')")
|
||||
assert(result.strip() == 'x\tUInt64')
|
||||
|
||||
result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')")
|
||||
assert(int(result) == 1)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
cluster.start()
|
||||
input("Cluster created, press any key to destroy...")
|
||||
|
@ -1038,3 +1038,15 @@ def test_signatures(started_cluster):
|
||||
result = instance.query(f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test.arrow', 'minio', 'minio123', 'Arrow')")
|
||||
assert(int(result) == 1)
|
||||
|
||||
|
||||
def test_insert_select_schema_inference(started_cluster):
|
||||
bucket = started_cluster.minio_bucket
|
||||
instance = started_cluster.instances["dummy"]
|
||||
|
||||
instance.query(f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test.arrow') select toUInt64(1) as x settings s3_truncate_on_insert=1")
|
||||
result = instance.query(f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test.arrow')")
|
||||
assert(result.strip() == 'x\tUInt64')
|
||||
|
||||
result = instance.query(f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test.arrow')")
|
||||
assert(int(result) == 1)
|
||||
|
||||
|
@ -0,0 +1,13 @@
|
||||
x UInt32
|
||||
y String
|
||||
d Date
|
||||
0 0 1970-01-01
|
||||
1 1 1970-01-02
|
||||
2 2 1970-01-03
|
||||
3 3 1970-01-04
|
||||
4 4 1970-01-05
|
||||
5 5 1970-01-06
|
||||
6 6 1970-01-07
|
||||
7 7 1970-01-08
|
||||
8 8 1970-01-09
|
||||
9 9 1970-01-10
|
@ -0,0 +1,5 @@
|
||||
drop table if exists test;
|
||||
create table test (x UInt32, y String, d Date) engine=Memory() as select number as x, toString(number) as y, toDate(number) as d from numbers(10);
|
||||
insert into table function file('data.native.zst') select * from test;
|
||||
desc file('data.native.zst');
|
||||
select * from file('data.native.zst');
|
@ -0,0 +1,2 @@
|
||||
1 aaa
|
||||
2 bbb
|
@ -0,0 +1,11 @@
|
||||
DROP TABLE IF EXISTS t_async_insert_table_function;
|
||||
|
||||
CREATE TABLE t_async_insert_table_function (id UInt32, s String) ENGINE = Memory;
|
||||
|
||||
SET async_insert = 1;
|
||||
|
||||
INSERT INTO function remote('127.0.0.1', currentDatabase(), t_async_insert_table_function) values (1, 'aaa') (2, 'bbb');
|
||||
|
||||
SELECT * FROM t_async_insert_table_function ORDER BY id;
|
||||
|
||||
DROP TABLE t_async_insert_table_function;
|
17
tests/queries/0_stateless/02229_client_stop_multiquery_in_SIGINT.sh
Executable file
17
tests/queries/0_stateless/02229_client_stop_multiquery_in_SIGINT.sh
Executable file
@ -0,0 +1,17 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
timeout -s INT 3s $CLICKHOUSE_CLIENT --max_block_size 1 -nm -q "
|
||||
SELECT sleep(1) FROM numbers(100) FORMAT Null;
|
||||
SELECT 'FAIL';
|
||||
"
|
||||
|
||||
timeout -s INT 3s $CLICKHOUSE_LOCAL --max_block_size 1 -nm -q "
|
||||
SELECT sleep(1) FROM numbers(100) FORMAT Null;
|
||||
SELECT 'FAIL';
|
||||
"
|
||||
|
||||
exit 0
|
@ -0,0 +1,32 @@
|
||||
CREATE TABLE default.data_02230_ttl
|
||||
(
|
||||
`date` Date,
|
||||
`key` Int32
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key
|
||||
TTL date + 14
|
||||
SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.null_02230_ttl
|
||||
(
|
||||
`date` Date,
|
||||
`key` Int32
|
||||
)
|
||||
ENGINE = Null
|
||||
CREATE TABLE default.data_02230_column_ttl
|
||||
(
|
||||
`date` Date,
|
||||
`value` Int32 TTL date + 7,
|
||||
`key` Int32
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key
|
||||
TTL date + 14
|
||||
SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.null_02230_column_ttl
|
||||
(
|
||||
`date` Date,
|
||||
`value` Int32,
|
||||
`key` Int32
|
||||
)
|
||||
ENGINE = Null
|
@ -0,0 +1,18 @@
|
||||
drop table if exists data_02230_ttl;
|
||||
drop table if exists null_02230_ttl;
|
||||
create table data_02230_ttl (date Date, key Int) Engine=MergeTree() order by key TTL date + 14;
|
||||
show create data_02230_ttl format TSVRaw;
|
||||
create table null_02230_ttl engine=Null() as data_02230_ttl;
|
||||
show create null_02230_ttl format TSVRaw;
|
||||
drop table data_02230_ttl;
|
||||
drop table null_02230_ttl;
|
||||
|
||||
drop table if exists data_02230_column_ttl;
|
||||
drop table if exists null_02230_column_ttl;
|
||||
create table data_02230_column_ttl (date Date, value Int TTL date + 7, key Int) Engine=MergeTree() order by key TTL date + 14;
|
||||
show create data_02230_column_ttl format TSVRaw;
|
||||
create table null_02230_column_ttl engine=Null() as data_02230_column_ttl;
|
||||
-- check that order of columns is the same
|
||||
show create null_02230_column_ttl format TSVRaw;
|
||||
drop table data_02230_column_ttl;
|
||||
drop table null_02230_column_ttl;
|
@ -85,6 +85,11 @@ Results for ThinkPad P15 are from <b>Mikhail Shiryaev</b>.<br/>
|
||||
Results for RockPi4 are from <b>Kirill Zholnay</b>.<br/>
|
||||
Results for Xeon 6266C are from <b>David in Shanghai</b>.<br/>
|
||||
Results for SSDNodes and Cavium are from <b>Lorenzo QXIP</b>.<br/>
|
||||
Results for AMD EPYC 7662 64-Core Processor are from <b>Evgeniy Kuts</b>.<br/>
|
||||
Results for scaleway GP1-S 8x x86 64bit 32GB ram 300gb NVMe are from <b>Dag Vilmar Tveit</b>.<br/>
|
||||
Results for scaleway GP1-M 16x x86 64bit 64GB ram 600gb NVMe are from <b>Dag Vilmar Tveit</b>.<br/>
|
||||
Results for Intel(R) Core(TM) i5-4440 CPU @ 3.10GHz are from <b>Peter, Chun-Sheng, Li</b>.<br/>
|
||||
Results for MacBook Pro M1 are from <b>Filatenkov Arthur</b>.<br/>
|
||||
</p>
|
||||
</div>
|
||||
</div>
|
||||
|
54
website/benchmark/hardware/results/amd_epyc_7662.json
Normal file
54
website/benchmark/hardware/results/amd_epyc_7662.json
Normal file
@ -0,0 +1,54 @@
|
||||
[
|
||||
{
|
||||
"system": "AMD EPYC 7662",
|
||||
"system_full": "AMD EPYC 7662 64-Core Processor",
|
||||
"time": "2022-01-26 11:28:55",
|
||||
"kind": "server",
|
||||
"result":
|
||||
[
|
||||
[0.001, 0.001, 0.001],
|
||||
[0.037, 0.019, 0.020],
|
||||
[0.082, 0.034, 0.026],
|
||||
[0.298, 0.045, 0.038],
|
||||
[0.424, 0.188, 0.178],
|
||||
[0.594, 0.229, 0.227],
|
||||
[0.037, 0.028, 0.032],
|
||||
[0.060, 0.028, 0.027],
|
||||
[0.496, 0.185, 0.192],
|
||||
[0.611, 0.210, 0.214],
|
||||
[0.400, 0.148, 0.137],
|
||||
[0.424, 0.155, 0.144],
|
||||
[0.639, 0.256, 0.239],
|
||||
[0.944, 0.404, 0.309],
|
||||
[0.699, 0.326, 0.288],
|
||||
[0.461, 0.221, 0.216],
|
||||
[1.176, 0.539, 0.561],
|
||||
[1.070, 0.410, 0.426],
|
||||
[2.080, 0.950, 0.866],
|
||||
[0.351, 0.066, 0.130],
|
||||
[3.248, 0.461, 0.313],
|
||||
[3.612, 0.261, 0.231],
|
||||
[6.720, 0.682, 0.671],
|
||||
[6.300, 0.517, 0.488],
|
||||
[0.982, 0.136, 0.125],
|
||||
[0.531, 0.112, 0.109],
|
||||
[1.006, 0.133, 0.118],
|
||||
[3.184, 0.324, 0.310],
|
||||
[2.799, 0.327, 0.308],
|
||||
[0.569, 0.492, 0.493],
|
||||
[0.900, 0.212, 0.221],
|
||||
[1.925, 0.353, 0.326],
|
||||
[2.489, 1.173, 1.248],
|
||||
[3.626, 0.990, 0.897],
|
||||
[3.743, 0.935, 0.915],
|
||||
[0.419, 0.311, 0.339],
|
||||
[0.278, 0.244, 0.236],
|
||||
[0.111, 0.099, 0.098],
|
||||
[0.139, 0.086, 0.084],
|
||||
[0.664, 0.520, 0.552],
|
||||
[0.072, 0.028, 0.036],
|
||||
[0.050, 0.031, 0.022],
|
||||
[0.005, 0.005, 0.011]
|
||||
]
|
||||
}
|
||||
]
|
54
website/benchmark/hardware/results/gp1_s_16x.json
Normal file
54
website/benchmark/hardware/results/gp1_s_16x.json
Normal file
@ -0,0 +1,54 @@
|
||||
[
|
||||
{
|
||||
"system": "scaleway GP1-S 8x x86",
|
||||
"system_full": "scaleway GP1-M 16x x86 64bit 64GB ram 600gb NVMe",
|
||||
"time": "2022-02-16 00:00:00",
|
||||
"kind": "cloud",
|
||||
"result":
|
||||
[
|
||||
[0.005, 0.005, 0.036],
|
||||
[0.039, 0.026, 0.026],
|
||||
[0.092, 0.046, 0.046],
|
||||
[0.172, 0.056, 0.055],
|
||||
[0.166, 0.126, 0.123],
|
||||
[0.364, 0.272, 0.265],
|
||||
[0.005, 0.006, 0.005],
|
||||
[0.028, 0.027, 0.029],
|
||||
[0.581, 0.49, 0.486],
|
||||
[0.69, 0.549, 0.553],
|
||||
[0.248, 0.178, 0.175],
|
||||
[0.266, 0.208, 0.208],
|
||||
[1.584, 1.017, 0.868],
|
||||
[1.717, 1.113, 1.145],
|
||||
[1.144, 1.084, 1.048],
|
||||
[0.991, 0.92, 0.895],
|
||||
[4.121, 2.639, 2.621],
|
||||
[1.447, 1.348, 1.354],
|
||||
[6.802, 6.466, 6.433],
|
||||
[0.142, 0.057, 0.052],
|
||||
[1.252, 0.743, 0.715],
|
||||
[1.389, 0.823, 0.791],
|
||||
[3.143, 2.225, 2.159],
|
||||
[1.795, 0.871, 0.837],
|
||||
[0.361, 0.236, 0.229],
|
||||
[0.264, 0.211, 0.214],
|
||||
[0.37, 0.24, 0.225],
|
||||
[1.449, 0.967, 0.876],
|
||||
[1.605, 1.206, 1.16 ],
|
||||
[3.412, 3.388, 3.397],
|
||||
[0.783, 0.628, 0.65 ],
|
||||
[1.419, 1.134, 1.112],
|
||||
[6.983, 6.843, 6.852],
|
||||
[5.466, 5.082, 4.955],
|
||||
[5.632, 4.972, 5.22 ],
|
||||
[1.639, 1.604, 1.571],
|
||||
[0.285, 0.298, 0.269],
|
||||
[0.115, 0.115, 0.101],
|
||||
[0.098, 0.1, 0.092],
|
||||
[0.563, 0.562, 0.512],
|
||||
[0.058, 0.039, 0.042],
|
||||
[0.039, 0.039, 0.025],
|
||||
[0.029, 0.012, 0.012]
|
||||
]
|
||||
}
|
||||
]
|
54
website/benchmark/hardware/results/gp1_s_8x.json
Normal file
54
website/benchmark/hardware/results/gp1_s_8x.json
Normal file
@ -0,0 +1,54 @@
|
||||
[
|
||||
{
|
||||
"system": "scaleway GP1-S 8x x86",
|
||||
"system_full": "scaleway GP1-S 8x x86 64bit 32GB ram 300gb NVMe",
|
||||
"time": "2022-02-16 00:00:00",
|
||||
"kind": "cloud",
|
||||
"result":
|
||||
[
|
||||
[0.026, 0.004, 0.004],
|
||||
[0.038, 0.026, 0.026],
|
||||
[0.071, 0.058, 0.059],
|
||||
[0.118, 0.072, 0.069],
|
||||
[0.190, 0.151, 0.155],
|
||||
[0.465, 0.438, 0.401],
|
||||
[0.002, 0.004, 0.004],
|
||||
[0.028, 0.029, 0.026],
|
||||
[0.751, 0.672, 0.676],
|
||||
[0.897, 0.845, 0.798],
|
||||
[0.291, 0.234, 0.254],
|
||||
[0.371, 0.297, 0.296],
|
||||
[1.208, 1.041, 1.005],
|
||||
[1.445, 1.400, 1.414],
|
||||
[1.406, 1.317, 1.342],
|
||||
[1.414, 1.242, 1.244],
|
||||
[4.179, 3.849, 3.878],
|
||||
[2.320, 2.275, 2.201],
|
||||
[7.499, 7.424, 7.196],
|
||||
[0.135, 0.077, 0.068],
|
||||
[1.465, 1.075, 1.063],
|
||||
[1.700, 1.221, 1.198],
|
||||
[3.731, 2.959, 2.905],
|
||||
[2.283, 1.401, 1.342],
|
||||
[0.474, 0.377, 0.367],
|
||||
[0.371, 0.314, 0.337],
|
||||
[0.483, 0.357, 0.356],
|
||||
[1.565, 1.194, 1.181],
|
||||
[2.226, 1.815, 1.746],
|
||||
[2.990, 2.971, 2.947],
|
||||
[1.003, 0.815, 0.842],
|
||||
[1.386, 1.127, 1.108],
|
||||
[8.174, 7.690, 7.735],
|
||||
[6.171, 5.802, 5.933],
|
||||
[6.201, 5.774, 5.972],
|
||||
[1.758, 1.642, 1.639],
|
||||
[0.288, 0.273, 0.253],
|
||||
[0.121, 0.125, 0.107],
|
||||
[0.096, 0.082, 0.088],
|
||||
[0.490, 0.461, 0.476],
|
||||
[0.041, 0.037, 0.035],
|
||||
[0.035, 0.031, 0.025],
|
||||
[0.008, 0.011, 0.015]
|
||||
]
|
||||
}
|
||||
]
|
54
website/benchmark/hardware/results/intel_core_i5_4440.json
Normal file
54
website/benchmark/hardware/results/intel_core_i5_4440.json
Normal file
@ -0,0 +1,54 @@
|
||||
[
|
||||
{
|
||||
"system": "Intel(R) Core(TM) i5-4440 CPU @ 3.10GHz",
|
||||
"system_full": "Intel(R) Core(TM) i5-4440 CPU @ 3.10GHz",
|
||||
"time": "2022-01-06 08:48:45",
|
||||
"kind": "server",
|
||||
"result":
|
||||
[
|
||||
[0.002, 0.001, 0.001],
|
||||
[0.136, 0.021, 0.020],
|
||||
[1.102, 0.061, 0.055],
|
||||
[2.669, 0.089, 0.084],
|
||||
[2.646, 0.198, 0.192],
|
||||
[4.018, 0.606, 0.600],
|
||||
[0.115, 0.034, 0.044],
|
||||
[0.210, 0.018, 0.018],
|
||||
[4.655, 1.002, 1.004],
|
||||
[6.715, 1.139, 1.150],
|
||||
[3.235, 0.351, 0.352],
|
||||
[3.850, 0.410, 0.408],
|
||||
[4.446, 1.579, 1.570],
|
||||
[7.112, 2.031, 2.061],
|
||||
[5.658, 1.812, 1.804],
|
||||
[3.528, 1.600, 1.599],
|
||||
[9.216, 5.029, 5.031],
|
||||
[7.023, 2.968, 3.362],
|
||||
[17.412, 9.705, 9.695],
|
||||
[2.717, 0.110, 0.100],
|
||||
[28.586, 1.907, 1.870],
|
||||
[34.064, 2.178, 2.172],
|
||||
[67.172, 5.105, 5.101],
|
||||
[79.885, 2.579, 2.540],
|
||||
[9.176, 0.572, 0.560],
|
||||
[4.050, 0.496, 0.492],
|
||||
[8.918, 0.575, 0.568],
|
||||
[28.731, 2.089, 2.058],
|
||||
[24.174, 2.956, 3.043],
|
||||
[5.103, 5.010, 5.007],
|
||||
[10.075, 1.188, 1.197],
|
||||
[18.485, 1.966, 1.954],
|
||||
[19.455, 10.855, 10.917],
|
||||
[31.320, 7.848, 7.831],
|
||||
[30.794, 7.871, 7.877],
|
||||
[3.360, 2.777, 2.778],
|
||||
[0.371, 0.166, 0.180],
|
||||
[0.259, 0.064, 0.083],
|
||||
[0.275, 0.060, 0.058],
|
||||
[1.024, 0.380, 0.378],
|
||||
[0.198, 0.025, 0.025],
|
||||
[0.162, 0.023, 0.015],
|
||||
[0.059, 0.006, 0.007]
|
||||
]
|
||||
}
|
||||
]
|
54
website/benchmark/hardware/results/macbook_pro_m1_2021.json
Normal file
54
website/benchmark/hardware/results/macbook_pro_m1_2021.json
Normal file
@ -0,0 +1,54 @@
|
||||
[
|
||||
{
|
||||
"system": "MacBook Pro M1",
|
||||
"system_full": "MacBook Pro M1 Max 16\" 2022, 64 GiB RAM, 1 TB SSD",
|
||||
"time": "2022-02-27 00:00:00",
|
||||
"kind": "laptop",
|
||||
"result":
|
||||
[
|
||||
[0.012, 0.001, 0.001],
|
||||
[0.096, 0.012, 0.010],
|
||||
[0.043, 0.022, 0.023],
|
||||
[0.063, 0.031, 0.030],
|
||||
[0.099, 0.070, 0.070],
|
||||
[0.229, 0.197, 0.195],
|
||||
[0.012, 0.001, 0.001],
|
||||
[0.027, 0.012, 0.011],
|
||||
[0.340, 0.301, 0.306],
|
||||
[0.439, 0.383, 0.386],
|
||||
[0.169, 0.134, 0.136],
|
||||
[0.197, 0.160, 0.162],
|
||||
[0.475, 0.435, 0.432],
|
||||
[0.615, 0.557, 0.553],
|
||||
[0.553, 0.502, 0.507],
|
||||
[0.490, 0.445, 0.439],
|
||||
[1.392, 1.260, 1.254],
|
||||
[0.865, 0.833, 0.835],
|
||||
[2.285, 2.180, 2.194],
|
||||
[0.064, 0.035, 0.033],
|
||||
[0.761, 0.650, 0.651],
|
||||
[0.867, 0.715, 0.718],
|
||||
[1.753, 1.478, 1.499],
|
||||
[1.037, 0.737, 0.735],
|
||||
[0.251, 0.201, 0.202],
|
||||
[0.208, 0.172, 0.174],
|
||||
[0.254, 0.202, 0.201],
|
||||
[0.733, 0.598, 0.603],
|
||||
[0.995, 0.882, 0.879],
|
||||
[0.562, 0.545, 0.545],
|
||||
[0.431, 0.371, 0.371],
|
||||
[0.586, 0.490, 0.490],
|
||||
[2.882, 2.664, 2.656],
|
||||
[2.255, 2.147, 2.146],
|
||||
[2.248, 2.137, 2.154],
|
||||
[0.659, 0.638, 0.631],
|
||||
[0.125, 0.108, 0.108],
|
||||
[0.070, 0.052, 0.052],
|
||||
[0.060, 0.042, 0.042],
|
||||
[0.250, 0.229, 0.228],
|
||||
[0.030, 0.013, 0.012],
|
||||
[0.026, 0.011, 0.010],
|
||||
[0.017, 0.003, 0.003]
|
||||
]
|
||||
}
|
||||
]
|
Loading…
Reference in New Issue
Block a user