Merge branch 'master' into fix-redundant-linux-docs

This commit is contained in:
mergify[bot] 2022-03-01 16:52:54 +00:00 committed by GitHub
commit 00afead7ec
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
50 changed files with 897 additions and 76 deletions

View File

@ -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,20 +29,60 @@ 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"
exit 1
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
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
if [[ ! -f $QUERIES_FILE ]]; then
@ -88,7 +124,12 @@ echo
cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do
sync
echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null
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,27 +145,45 @@ echo
echo "Benchmark complete. System info:"
echo
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
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
echo '----CPU-------------------------'
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
lscpu
echo '----Block Devices---------------'
lsblk
echo '----Disk Free and Total--------'
df -h .
echo '----Memory Free and Total-------'
free -h
echo '----Physical Memory Amount------'
cat /proc/meminfo | grep MemTotal
echo '----RAID Info-------------------'
cat /proc/mdstat
#echo '----PCI-------------------------'
#lspci
#echo '----All Hardware Info-----------'
#lshw
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
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
echo '----CPU-------------------------'
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
lscpu
echo '----Block Devices---------------'
lsblk
echo '----Disk Free and Total--------'
df -h .
echo '----Memory Free and Total-------'
free -h
echo '----Physical Memory Amount------'
cat /proc/meminfo | grep MemTotal
echo '----RAID Info-------------------'
cat /proc/mdstat
#echo '----PCI-------------------------'
#lspci
#echo '----All Hardware Info-----------'
#lshw
echo '--------------------------------'
fi
echo

View File

@ -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
```

View File

@ -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

View File

@ -1 +0,0 @@
../../../en/faq/integration/file-export.md

View 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).

View File

@ -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
```

View File

@ -1 +0,0 @@
../../../../en/sql-reference/statements/alter/role.md

View 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'] [,...]
```

View File

@ -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);
}
@ -243,22 +243,22 @@ ClientBase::ClientBase() = default;
void ClientBase::setupSignalHandler()
{
exit_on_signal.test_and_set();
exit_on_signal.test_and_set();
struct sigaction new_act;
memset(&new_act, 0, sizeof(new_act));
struct sigaction new_act;
memset(&new_act, 0, sizeof(new_act));
new_act.sa_handler = interruptSignalHandler;
new_act.sa_flags = 0;
new_act.sa_handler = interruptSignalHandler;
new_act.sa_flags = 0;
#if defined(OS_DARWIN)
sigemptyset(&new_act.sa_mask);
#else
if (sigemptyset(&new_act.sa_mask))
if (sigemptyset(&new_act.sa_mask))
throw Exception(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler.");
#endif
if (sigaction(SIGINT, &new_act, nullptr))
if (sigaction(SIGINT, &new_act, nullptr))
throw Exception(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler.");
}
@ -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;

View File

@ -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;
};
}

View File

@ -184,7 +184,10 @@ 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);
query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames());
/// 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();

View File

@ -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();
}

View File

@ -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());

View File

@ -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;
}

View File

@ -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)

View File

@ -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);
};

View File

@ -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
{

View File

@ -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;

View File

@ -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>

View File

@ -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;
}

View File

@ -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>;

View File

@ -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})
target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms)
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 ()

View 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

View 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

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -41,6 +41,7 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context
return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context);
}
return parseColumnsListFromString(structure, context);
}

View File

@ -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,

View File

@ -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

View File

@ -31,6 +31,10 @@ void registerTableFunctions()
registerTableFunctionHDFSCluster(factory);
#endif
#if USE_HIVE
registerTableFunctionHive(factory);
#endif
registerTableFunctionODBC(factory);
registerTableFunctionJDBC(factory);

View File

@ -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);

View File

@ -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")

View 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-----

View File

@ -0,0 +1,6 @@
-----BEGIN PRIVATE KEY-----
MIG2AgEAMBAGByqGSM49AgEGBSuBBAAiBIGeMIGbAgEBBDAJbfB78wfRHn5A4x3e
EAqrFk/hbBD+c8snbFgjQqxg4qTcp154Rc01B9V0US27MJuhZANiAASCg9j+vz3/
zfMfOaluwD6Rbdh+VVC0pc7wqXuSLmr6/WpS7fGavge203G8cs4J8rv3wowaD3L4
07nLgPgPZmAsbGQ23xfJa+GRt7fEp0ZKoGmmVstP4z0W2ZDq3PPMUlU=
-----END PRIVATE KEY-----

View File

@ -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

View File

@ -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...")

View File

@ -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)

View File

@ -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

View File

@ -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');

View File

@ -0,0 +1,2 @@
1 aaa
2 bbb

View File

@ -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;

View 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

View File

@ -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

View File

@ -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;

View File

@ -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>

View 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]
]
}
]

View 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]
]
}
]

View 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]
]
}
]

View 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]
]
}
]

View 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]
]
}
]