mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge branch 'master' into simplify-test
This commit is contained in:
commit
692afc6bf6
2
contrib/base64
vendored
2
contrib/base64
vendored
@ -1 +1 @@
|
||||
Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46
|
||||
Subproject commit 8628e258090f9eb76d90ac3c91e1ab4690e9aa11
|
@ -52,9 +52,9 @@ URI::URI(const std::string & uri_)
|
||||
has_version_id = true;
|
||||
}
|
||||
|
||||
/// Poco::URI will ignore '?' when parsing the path, but if there is a vestionId in the http parameter,
|
||||
/// Poco::URI will ignore '?' when parsing the path, but if there is a versionId in the http parameter,
|
||||
/// '?' can not be used as a wildcard, otherwise it will be ambiguous.
|
||||
/// If no "vertionId" in the http parameter, '?' can be used as a wildcard.
|
||||
/// If no "versionId" in the http parameter, '?' can be used as a wildcard.
|
||||
/// It is necessary to encode '?' to avoid deletion during parsing path.
|
||||
if (!has_version_id && uri_.find('?') != String::npos)
|
||||
{
|
||||
|
@ -16,7 +16,6 @@ NamesAndTypesList MetricLogElement::getNamesAndTypes()
|
||||
columns_with_type_and_name.emplace_back("event_date", std::make_shared<DataTypeDate>());
|
||||
columns_with_type_and_name.emplace_back("event_time", std::make_shared<DataTypeDateTime>());
|
||||
columns_with_type_and_name.emplace_back("event_time_microseconds", std::make_shared<DataTypeDateTime64>(6));
|
||||
columns_with_type_and_name.emplace_back("milliseconds", std::make_shared<DataTypeUInt64>());
|
||||
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
{
|
||||
@ -45,7 +44,6 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const
|
||||
columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
|
||||
columns[column_idx++]->insert(event_time);
|
||||
columns[column_idx++]->insert(event_time_microseconds);
|
||||
columns[column_idx++]->insert(milliseconds);
|
||||
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
columns[column_idx++]->insert(profile_events[i]);
|
||||
@ -96,7 +94,6 @@ void MetricLog::metricThreadFunction()
|
||||
MetricLogElement elem;
|
||||
elem.event_time = std::chrono::system_clock::to_time_t(current_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.milliseconds = timeInMilliseconds(current_time) - timeInSeconds(current_time) * 1000;
|
||||
|
||||
elem.profile_events.resize(ProfileEvents::end());
|
||||
for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i)
|
||||
|
@ -22,7 +22,6 @@ struct MetricLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
Decimal64 event_time_microseconds{};
|
||||
UInt64 milliseconds{};
|
||||
|
||||
std::vector<ProfileEvents::Count> profile_events;
|
||||
std::vector<CurrentMetrics::Metric> current_metrics;
|
||||
|
@ -251,15 +251,26 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
size_t total_number_of_rows = 0;
|
||||
size_t total_number_of_parts = 0;
|
||||
|
||||
size_t total_number_of_tables_system = 0;
|
||||
|
||||
size_t total_number_of_bytes_system = 0;
|
||||
size_t total_number_of_rows_system = 0;
|
||||
size_t total_number_of_parts_system = 0;
|
||||
|
||||
for (const auto & db : databases)
|
||||
{
|
||||
/// Check if database can contain MergeTree tables
|
||||
if (!db.second->canContainMergeTreeTables())
|
||||
continue;
|
||||
|
||||
bool is_system = db.first == DatabaseCatalog::SYSTEM_DATABASE;
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
|
||||
{
|
||||
++total_number_of_tables;
|
||||
if (is_system)
|
||||
++total_number_of_tables_system;
|
||||
|
||||
const auto & table = iterator->table();
|
||||
if (!table)
|
||||
continue;
|
||||
@ -269,9 +280,21 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
|
||||
calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first);
|
||||
total_number_of_bytes += table_merge_tree->totalBytes(settings).value();
|
||||
total_number_of_rows += table_merge_tree->totalRows(settings).value();
|
||||
total_number_of_parts += table_merge_tree->getActivePartsCount();
|
||||
|
||||
size_t bytes = table_merge_tree->totalBytes(settings).value();
|
||||
size_t rows = table_merge_tree->totalRows(settings).value();
|
||||
size_t parts = table_merge_tree->getActivePartsCount();
|
||||
|
||||
total_number_of_bytes += bytes;
|
||||
total_number_of_rows += rows;
|
||||
total_number_of_parts += parts;
|
||||
|
||||
if (is_system)
|
||||
{
|
||||
total_number_of_bytes_system += bytes;
|
||||
total_number_of_rows_system += rows;
|
||||
total_number_of_parts_system += parts;
|
||||
}
|
||||
}
|
||||
|
||||
if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast<StorageReplicatedMergeTree *>(table.get()))
|
||||
@ -325,6 +348,12 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." };
|
||||
new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family."
|
||||
" Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." };
|
||||
|
||||
new_values["NumberOfTablesSystem"] = { total_number_of_tables_system, "Total number of tables in the system database on the server stored in tables of MergeTree family."};
|
||||
|
||||
new_values["TotalBytesOfMergeTreeTablesSystem"] = { total_number_of_bytes_system, "Total amount of bytes (compressed, including data and indices) stored in tables of MergeTree family in the system database." };
|
||||
new_values["TotalRowsOfMergeTreeTablesSystem"] = { total_number_of_rows_system, "Total amount of rows (records) stored in tables of MergeTree family in the system database." };
|
||||
new_values["TotalPartsOfMergeTreeTablesSystem"] = { total_number_of_parts_system, "Total amount of data parts in tables of MergeTree family in the system database." };
|
||||
}
|
||||
|
||||
#if USE_NURAFT
|
||||
|
@ -36,7 +36,6 @@ NamesAndTypesList TextLogElement::getNamesAndTypes()
|
||||
{"event_date", std::make_shared<DataTypeDate>()},
|
||||
{"event_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
|
||||
{"microseconds", std::make_shared<DataTypeUInt32>()},
|
||||
|
||||
{"thread_name", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"thread_id", std::make_shared<DataTypeUInt64>()},
|
||||
@ -62,7 +61,6 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const
|
||||
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
|
||||
columns[i++]->insert(event_time);
|
||||
columns[i++]->insert(event_time_microseconds);
|
||||
columns[i++]->insert(microseconds);
|
||||
|
||||
columns[i++]->insertData(thread_name.data(), thread_name.size());
|
||||
columns[i++]->insert(thread_id);
|
||||
|
@ -14,7 +14,6 @@ struct TextLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
Decimal64 event_time_microseconds{};
|
||||
UInt32 microseconds{};
|
||||
|
||||
String thread_name;
|
||||
UInt64 thread_id{};
|
||||
|
@ -117,7 +117,6 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
|
||||
|
||||
elem.event_time = msg_ext.time_seconds;
|
||||
elem.event_time_microseconds = msg_ext.time_in_microseconds;
|
||||
elem.microseconds = msg_ext.time_microseconds;
|
||||
|
||||
elem.thread_name = getThreadName();
|
||||
elem.thread_id = msg_ext.thread_id;
|
||||
|
@ -55,7 +55,7 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" 2>&1 | grep -F -c "CANNOT_SKIP_UNKNOWN_FIELD"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNames" 2>&1 | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" 2>&1 | grep -F -c "CANNOT_SKIP_UNKNOWN_FIELD"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||
@ -63,8 +63,8 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -c "INCORRECT_DATA"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" 2>&1 | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -c "INCORRECT_DATA"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -c "INCORRECT_DATA"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" 2>&1 | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -c "INCORRECT_DATA"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02102"
|
||||
|
Loading…
Reference in New Issue
Block a user