mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' into sanych73-prepared_statements
This commit is contained in:
commit
f38855fe61
@ -310,7 +310,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
/// Initialize DateLUT early, to not interfere with running time of first query.
|
||||
LOG_DEBUG(log, "Initializing DateLUT.");
|
||||
DateLUT::instance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone `" << DateLUT::instance().getTimeZone() << "'.");
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '" << DateLUT::instance().getTimeZone() << "'.");
|
||||
|
||||
/// Directory with temporary data for processing of heavy queries.
|
||||
{
|
||||
|
@ -527,7 +527,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig(bool allow_zk_includes
|
||||
XMLDocumentPtr config_xml = processConfig(&has_zk_includes);
|
||||
|
||||
if (has_zk_includes && !allow_zk_includes)
|
||||
throw Poco::Exception("Error while loading config `" + path + "': from_zk includes are not allowed!");
|
||||
throw Poco::Exception("Error while loading config '" + path + "': from_zk includes are not allowed!");
|
||||
|
||||
ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(config_xml));
|
||||
|
||||
|
@ -87,7 +87,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
ConfigProcessor::LoadedConfig loaded_config;
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Loading config `" << path << "'");
|
||||
LOG_DEBUG(log, "Loading config '" << path << "'");
|
||||
|
||||
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
|
||||
if (loaded_config.has_zk_includes)
|
||||
@ -102,7 +102,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
if (throw_on_error)
|
||||
throw;
|
||||
|
||||
tryLogCurrentException(log, "ZooKeeper error when loading config from `" + path + "'");
|
||||
tryLogCurrentException(log, "ZooKeeper error when loading config from '" + path + "'");
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
@ -110,7 +110,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
if (throw_on_error)
|
||||
throw;
|
||||
|
||||
tryLogCurrentException(log, "Error loading config from `" + path + "'");
|
||||
tryLogCurrentException(log, "Error loading config from '" + path + "'");
|
||||
return;
|
||||
}
|
||||
config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir);
|
||||
@ -134,7 +134,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw;
|
||||
tryLogCurrentException(log, "Error updating configuration from `" + path + "' config.");
|
||||
tryLogCurrentException(log, "Error updating configuration from '" + path + "' config.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
30
dbms/src/Common/IPv6ToBinary.cpp
Normal file
30
dbms/src/Common/IPv6ToBinary.cpp
Normal file
@ -0,0 +1,30 @@
|
||||
#include "IPv6ToBinary.h"
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::array<char, 16> IPv6ToBinary(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
std::array<char, 16> res;
|
||||
|
||||
if (Poco::Net::IPAddress::IPv6 == address.family())
|
||||
{
|
||||
memcpy(res.data(), address.addr(), 16);
|
||||
}
|
||||
else if (Poco::Net::IPAddress::IPv4 == address.family())
|
||||
{
|
||||
/// Convert to IPv6-mapped address.
|
||||
memset(res.data(), 0, 10);
|
||||
res[10] = '\xFF';
|
||||
res[11] = '\xFF';
|
||||
memcpy(&res[12], address.addr(), 4);
|
||||
}
|
||||
else
|
||||
memset(res.data(), 0, 16);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
11
dbms/src/Common/IPv6ToBinary.h
Normal file
11
dbms/src/Common/IPv6ToBinary.h
Normal file
@ -0,0 +1,11 @@
|
||||
#include <array>
|
||||
|
||||
namespace Poco { namespace Net { class IPAddress; }}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Convert IP address to 16-byte array with IPv6 data (big endian). If it's an IPv4, map it to IPv6.
|
||||
std::array<char, 16> IPv6ToBinary(const Poco::Net::IPAddress & address);
|
||||
|
||||
}
|
@ -36,7 +36,7 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coor
|
||||
|
||||
zkutil::ZooKeeperPtr zookeeper = get_zookeeper();
|
||||
if (!zookeeper)
|
||||
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
|
||||
throw DB::Exception("Could not get znode: '" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
|
||||
|
||||
for (const auto & invalidated_path : invalidated_paths)
|
||||
path_to_cached_znode.erase(invalidated_path);
|
||||
|
@ -69,5 +69,5 @@ IRegionsHierarchyDataSourcePtr RegionsHierarchiesDataProvider::getHierarchySourc
|
||||
return std::make_shared<RegionsHierarchyDataSource>(hierarchy_file_path);
|
||||
}
|
||||
|
||||
throw Poco::Exception("Regions hierarchy `" + name + "` not found");
|
||||
throw Poco::Exception("Regions hierarchy '" + name + "' not found");
|
||||
}
|
||||
|
@ -197,7 +197,7 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & config,
|
||||
for (const auto & key : config_keys)
|
||||
{
|
||||
if (key.find('.') != String::npos)
|
||||
throw Exception("Cluster names with dots are not supported: `" + key + "`", ErrorCodes::SYNTAX_ERROR);
|
||||
throw Exception("Cluster names with dots are not supported: '" + key + "'", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
impl.emplace(key, std::make_shared<Cluster>(config, settings, config_name + "." + key));
|
||||
}
|
||||
|
@ -133,14 +133,14 @@ static void validateUpdateColumns(
|
||||
for (const auto & col : storage->getColumns().getMaterialized())
|
||||
{
|
||||
if (col.name == column_name)
|
||||
throw Exception("Cannot UPDATE materialized column `" + column_name + "`", ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
throw Exception("Cannot UPDATE materialized column " + backQuote(column_name), ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
}
|
||||
|
||||
throw Exception("There is no column `" + column_name + "` in table", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
throw Exception("There is no column " + backQuote(column_name) + " in table", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
}
|
||||
|
||||
if (key_columns.count(column_name))
|
||||
throw Exception("Cannot UPDATE key column `" + column_name + "`", ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
throw Exception("Cannot UPDATE key column " + backQuote(column_name), ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
|
||||
auto materialized_it = column_to_affected_materialized.find(column_name);
|
||||
if (materialized_it != column_to_affected_materialized.end())
|
||||
@ -148,8 +148,8 @@ static void validateUpdateColumns(
|
||||
for (const String & materialized : materialized_it->second)
|
||||
{
|
||||
if (key_columns.count(materialized))
|
||||
throw Exception("Updated column `" + column_name + "` affects MATERIALIZED column `"
|
||||
+ materialized + "`, which is a key column. Cannot UPDATE it.",
|
||||
throw Exception("Updated column " + backQuote(column_name) + " affects MATERIALIZED column "
|
||||
+ backQuote(materialized) + ", which is a key column. Cannot UPDATE it.",
|
||||
ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
}
|
||||
}
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/IPv6ToBinary.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -7,11 +9,10 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/ProfileEventsExt.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <array>
|
||||
|
||||
@ -44,11 +45,11 @@ Block QueryLogElement::createBlock()
|
||||
{std::make_shared<DataTypeUInt8>(), "is_initial_query"},
|
||||
{std::make_shared<DataTypeString>(), "user"},
|
||||
{std::make_shared<DataTypeString>(), "query_id"},
|
||||
{std::make_shared<DataTypeFixedString>(16), "address"},
|
||||
{DataTypeFactory::instance().get("IPv6"), "address"},
|
||||
{std::make_shared<DataTypeUInt16>(), "port"},
|
||||
{std::make_shared<DataTypeString>(), "initial_user"},
|
||||
{std::make_shared<DataTypeString>(), "initial_query_id"},
|
||||
{std::make_shared<DataTypeFixedString>(16), "initial_address"},
|
||||
{DataTypeFactory::instance().get("IPv6"), "initial_address"},
|
||||
{std::make_shared<DataTypeUInt16>(), "initial_port"},
|
||||
{std::make_shared<DataTypeUInt8>(), "interface"},
|
||||
{std::make_shared<DataTypeString>(), "os_user"},
|
||||
@ -73,29 +74,6 @@ Block QueryLogElement::createBlock()
|
||||
}
|
||||
|
||||
|
||||
static std::array<char, 16> IPv6ToBinary(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
std::array<char, 16> res;
|
||||
|
||||
if (Poco::Net::IPAddress::IPv6 == address.family())
|
||||
{
|
||||
memcpy(res.data(), address.addr(), 16);
|
||||
}
|
||||
else if (Poco::Net::IPAddress::IPv4 == address.family())
|
||||
{
|
||||
/// Convert to IPv6-mapped address.
|
||||
memset(res.data(), 0, 10);
|
||||
res[10] = '\xFF';
|
||||
res[11] = '\xFF';
|
||||
memcpy(&res[12], address.addr(), 4);
|
||||
}
|
||||
else
|
||||
memset(res.data(), 0, 16);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void QueryLogElement::appendToBlock(Block & block) const
|
||||
{
|
||||
MutableColumns columns = block.mutateColumns();
|
||||
|
@ -6,8 +6,8 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/ProfileEventsExt.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
@ -44,11 +44,11 @@ Block QueryThreadLogElement::createBlock()
|
||||
{std::make_shared<DataTypeUInt8>(), "is_initial_query"},
|
||||
{std::make_shared<DataTypeString>(), "user"},
|
||||
{std::make_shared<DataTypeString>(), "query_id"},
|
||||
{std::make_shared<DataTypeFixedString>(16), "address"},
|
||||
{DataTypeFactory::instance().get("IPv6"), "address"},
|
||||
{std::make_shared<DataTypeUInt16>(), "port"},
|
||||
{std::make_shared<DataTypeString>(), "initial_user"},
|
||||
{std::make_shared<DataTypeString>(), "initial_query_id"},
|
||||
{std::make_shared<DataTypeFixedString>(16), "initial_address"},
|
||||
{DataTypeFactory::instance().get("IPv6"), "initial_address"},
|
||||
{std::make_shared<DataTypeUInt16>(), "initial_port"},
|
||||
{std::make_shared<DataTypeUInt8>(), "interface"},
|
||||
{std::make_shared<DataTypeString>(), "os_user"},
|
||||
|
@ -274,7 +274,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
|
||||
});
|
||||
|
||||
if (insert_it == indices_description.indices.end())
|
||||
throw Exception("Wrong index name. Cannot find index `" + after_index_name + "` to insert after.",
|
||||
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
++insert_it;
|
||||
@ -296,7 +296,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
|
||||
{
|
||||
if (if_exists)
|
||||
return;
|
||||
throw Exception("Wrong index name. Cannot find index `" + index_name + "` to drop.",
|
||||
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
@ -62,7 +62,7 @@ void MergeTreeBloomFilterIndexGranule::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
if (empty())
|
||||
throw Exception(
|
||||
"Attempt to write empty minmax index `" + index.name + "`", ErrorCodes::LOGICAL_ERROR);
|
||||
"Attempt to write empty minmax index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (const auto & bloom_filter : bloom_filters)
|
||||
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), index.bloom_filter_size);
|
||||
@ -703,7 +703,7 @@ std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Unknown index type: `" + node->name + "`.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Unknown index type: " + backQuote(node->name), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -151,7 +151,7 @@ MergeTreeData::MergeTreeData(
|
||||
catch (Exception & e)
|
||||
{
|
||||
/// Better error message.
|
||||
e.addMessage("(while initializing MergeTree partition key from date column `" + date_column_name + "`)");
|
||||
e.addMessage("(while initializing MergeTree partition key from date column " + backQuote(date_column_name) + ")");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -381,7 +381,7 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns(
|
||||
|
||||
if (indices_names.find(new_indices.back()->name) != indices_names.end())
|
||||
throw Exception(
|
||||
"Index with name `" + new_indices.back()->name + "` already exsists",
|
||||
"Index with name " + backQuote(new_indices.back()->name) + " already exsists",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(index_decl->expr->clone());
|
||||
@ -1476,12 +1476,12 @@ void MergeTreeData::alterDataPart(
|
||||
exception_message << ", ";
|
||||
if (forbidden_because_of_modify)
|
||||
{
|
||||
exception_message << "from `" << from_to.first << "' to `" << from_to.second << "'";
|
||||
exception_message << "from " << backQuote(from_to.first) << " to " << backQuote(from_to.second);
|
||||
first = false;
|
||||
}
|
||||
else if (from_to.second.empty())
|
||||
{
|
||||
exception_message << "`" << from_to.first << "'";
|
||||
exception_message << backQuote(from_to.first);
|
||||
first = false;
|
||||
}
|
||||
}
|
||||
|
@ -999,7 +999,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
{
|
||||
if (!Poco::File(part->getFullPath() + index->getFileName() + ".idx").exists())
|
||||
{
|
||||
LOG_DEBUG(log, "File for index `" << index->name << "` does not exist. Skipping it.");
|
||||
LOG_DEBUG(log, "File for index " << backQuote(index->name) << " does not exist. Skipping it.");
|
||||
return ranges;
|
||||
}
|
||||
|
||||
@ -1054,7 +1054,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
last_index_mark = index_range.end - 1;
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Index `" << index->name << "` has dropped " << granules_dropped << " granules.");
|
||||
LOG_DEBUG(log, "Index " << backQuote(index->name) << " has dropped " << granules_dropped << " granules.");
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ void MergeTreeMinMaxGranule::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
if (empty())
|
||||
throw Exception(
|
||||
"Attempt to write empty minmax index `" + index.name + "`", ErrorCodes::LOGICAL_ERROR);
|
||||
"Attempt to write empty minmax index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (size_t i = 0; i < index.columns.size(); ++i)
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ void MergeTreeSetIndexGranule::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
if (empty())
|
||||
throw Exception(
|
||||
"Attempt to write empty set index `" + index.name + "`", ErrorCodes::LOGICAL_ERROR);
|
||||
"Attempt to write empty set index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
|
||||
|
||||
|
@ -38,7 +38,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
|
||||
const auto & assignment = assignment_ast->as<ASTAssignment &>();
|
||||
auto insertion = res.column_to_update_expression.emplace(assignment.column_name, assignment.expression);
|
||||
if (!insertion.second)
|
||||
throw Exception("Multiple assignments in the single statement to column `" + assignment.column_name + "`",
|
||||
throw Exception("Multiple assignments in the single statement to column " + backQuote(assignment.column_name),
|
||||
ErrorCodes::MULTIPLE_ASSIGNMENTS_TO_COLUMN);
|
||||
}
|
||||
return res;
|
||||
|
@ -1061,8 +1061,8 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
|
||||
FutureMergedMutatedPart future_merged_part(parts);
|
||||
if (future_merged_part.name != entry.new_part_name)
|
||||
{
|
||||
throw Exception("Future merged part name `" + future_merged_part.name + "` differs from part name in log entry: `"
|
||||
+ entry.new_part_name + "`", ErrorCodes::BAD_DATA_PART_NAME);
|
||||
throw Exception("Future merged part name " + backQuote(future_merged_part.name) + " differs from part name in log entry: "
|
||||
+ backQuote(entry.new_part_name), ErrorCodes::BAD_DATA_PART_NAME);
|
||||
}
|
||||
|
||||
MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(database_name, table_name, future_merged_part);
|
||||
|
@ -1,12 +1,14 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Storages/System/StorageSystemProcesses.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ProfileEventsExt.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/IPv6ToBinary.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
@ -21,12 +23,12 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
|
||||
|
||||
{"user", std::make_shared<DataTypeString>()},
|
||||
{"query_id", std::make_shared<DataTypeString>()},
|
||||
{"address", std::make_shared<DataTypeString>()},
|
||||
{"address", DataTypeFactory::instance().get("IPv6")},
|
||||
{"port", std::make_shared<DataTypeUInt16>()},
|
||||
|
||||
{"initial_user", std::make_shared<DataTypeString>()},
|
||||
{"initial_query_id", std::make_shared<DataTypeString>()},
|
||||
{"initial_address", std::make_shared<DataTypeString>()},
|
||||
{"initial_address", DataTypeFactory::instance().get("IPv6")},
|
||||
{"initial_port", std::make_shared<DataTypeUInt16>()},
|
||||
|
||||
{"interface", std::make_shared<DataTypeUInt8>()},
|
||||
@ -34,10 +36,10 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
|
||||
{"os_user", std::make_shared<DataTypeString>()},
|
||||
{"client_hostname", std::make_shared<DataTypeString>()},
|
||||
{"client_name", std::make_shared<DataTypeString>()},
|
||||
{"client_revision", std::make_shared<DataTypeUInt64>()},
|
||||
{"client_version_major", std::make_shared<DataTypeUInt64>()},
|
||||
{"client_version_minor", std::make_shared<DataTypeUInt64>()},
|
||||
{"client_version_patch", std::make_shared<DataTypeUInt64>()},
|
||||
{"client_revision", std::make_shared<DataTypeUInt64>()},
|
||||
|
||||
{"http_method", std::make_shared<DataTypeUInt8>()},
|
||||
{"http_user_agent", std::make_shared<DataTypeString>()},
|
||||
@ -71,26 +73,34 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, const Contex
|
||||
for (const auto & process : info)
|
||||
{
|
||||
size_t i = 0;
|
||||
|
||||
res_columns[i++]->insert(process.client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY);
|
||||
|
||||
res_columns[i++]->insert(process.client_info.current_user);
|
||||
res_columns[i++]->insert(process.client_info.current_query_id);
|
||||
res_columns[i++]->insert(process.client_info.current_address.host().toString());
|
||||
res_columns[i++]->insertData(IPv6ToBinary(process.client_info.current_address.host()).data(), 16);
|
||||
res_columns[i++]->insert(process.client_info.current_address.port());
|
||||
|
||||
res_columns[i++]->insert(process.client_info.initial_user);
|
||||
res_columns[i++]->insert(process.client_info.initial_query_id);
|
||||
res_columns[i++]->insert(process.client_info.initial_address.host().toString());
|
||||
res_columns[i++]->insertData(IPv6ToBinary(process.client_info.initial_address.host()).data(), 16);
|
||||
res_columns[i++]->insert(process.client_info.initial_address.port());
|
||||
|
||||
res_columns[i++]->insert(UInt64(process.client_info.interface));
|
||||
|
||||
res_columns[i++]->insert(process.client_info.os_user);
|
||||
res_columns[i++]->insert(process.client_info.client_hostname);
|
||||
res_columns[i++]->insert(process.client_info.client_name);
|
||||
res_columns[i++]->insert(process.client_info.client_revision);
|
||||
res_columns[i++]->insert(process.client_info.client_version_major);
|
||||
res_columns[i++]->insert(process.client_info.client_version_minor);
|
||||
res_columns[i++]->insert(process.client_info.client_version_patch);
|
||||
res_columns[i++]->insert(process.client_info.client_revision);
|
||||
|
||||
res_columns[i++]->insert(UInt64(process.client_info.http_method));
|
||||
res_columns[i++]->insert(process.client_info.http_user_agent);
|
||||
|
||||
res_columns[i++]->insert(process.client_info.quota_key);
|
||||
|
||||
res_columns[i++]->insert(process.elapsed_seconds);
|
||||
res_columns[i++]->insert(process.is_cancelled);
|
||||
res_columns[i++]->insert(process.read_rows);
|
||||
|
@ -116,7 +116,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co
|
||||
}
|
||||
|
||||
if (columns.empty())
|
||||
throw Exception("MySQL table `" + database_name + "`.`" + table_name + "` doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception("MySQL table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
auto res = StorageMySQL::create(
|
||||
table_name,
|
||||
|
52
dbms/tests/performance/number_formatting_formats.xml
Normal file
52
dbms/tests/performance/number_formatting_formats.xml
Normal file
@ -0,0 +1,52 @@
|
||||
<test>
|
||||
<name>number_formatting_formats</name>
|
||||
<type>loop</type>
|
||||
|
||||
<create_query>CREATE TABLE IF NOT EXISTS table_{format} (x UInt64) ENGINE = File({format})</create_query>
|
||||
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<iterations>5</iterations>
|
||||
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
|
||||
</all_of>
|
||||
<any_of>
|
||||
<iterations>100</iterations>
|
||||
<total_time_ms>60000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<main_metric>
|
||||
<min_time/>
|
||||
</main_metric>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>format</name>
|
||||
<values>
|
||||
<value>TabSeparated</value>
|
||||
<value>CSV</value>
|
||||
<value>Values</value>
|
||||
<value>JSON</value>
|
||||
<value>JSONCompact</value>
|
||||
<value>JSONEachRow</value>
|
||||
<value>TSKV</value>
|
||||
<value>RowBinary</value>
|
||||
<value>Native</value>
|
||||
<value>XML</value>
|
||||
<value>Parquet</value>
|
||||
<value>ODBCDriver2</value>
|
||||
<value>Null</value>
|
||||
<value>MySQLWire</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<preconditions>
|
||||
<table_exists>test.hits</table_exists>
|
||||
</preconditions>
|
||||
|
||||
<query>INSERT INTO table_{format} SELECT number FROM numbers(100000000)</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS table_{format}</drop_query>
|
||||
|
||||
</test>
|
@ -19,6 +19,9 @@
|
||||
<min_time/>
|
||||
</main_metric>
|
||||
|
||||
<settings>
|
||||
<output_format_pretty_max_rows>1000000</output_format_pretty_max_rows>
|
||||
</settings>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
@ -26,10 +29,7 @@
|
||||
<values>
|
||||
<value>TabSeparated</value>
|
||||
<value>TabSeparatedRaw</value>
|
||||
<value>TabSeparatedWithNames</value>
|
||||
<value>TabSeparatedWithNamesAndTypes</value>
|
||||
<value>CSV</value>
|
||||
<value>CSVWithNames</value>
|
||||
<value>Values</value>
|
||||
<value>Vertical</value>
|
||||
<value>JSON</value>
|
||||
@ -44,6 +44,8 @@
|
||||
<value>RowBinary</value>
|
||||
<value>Native</value>
|
||||
<value>XML</value>
|
||||
<value>ODBCDriver2</value>
|
||||
<value>MySQLWire</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
@ -235,6 +235,13 @@ To check whether ClickHouse can use the index when running a query, use the sett
|
||||
|
||||
The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date.
|
||||
|
||||
### Use of Index for Partially-Monotonic Primary Keys
|
||||
|
||||
Consider, for example, the days of the month. They are the [monotonic sequence](https://en.wikipedia.org/wiki/Monotonic_function) inside one month, but they are not monotonic for a more extended period. This is the partially-monotonic sequence. If a user creates the table with such partially-monotonic primary key, ClickHouse creates a sparse index as usual. When a user selects data from such a table, ClickHouse analyzes query conditions. If the user wants to get data between two marks of the index and both this marks are within one month, ClickHouse can use the index in this particular case because it can calculate the distance between parameters of query and index marks.
|
||||
|
||||
ClickHouse cannot use an index if the values of the primary key on the query parameters range don't represent the monotonic sequence. In this case, ClickHouse uses full scan method.
|
||||
|
||||
ClickHouse uses this logic not only for days of month sequences but for any primary key which represents a partially-monotonic sequence.
|
||||
|
||||
### Data Skipping Indices (Experimental)
|
||||
|
||||
|
@ -1,41 +1,41 @@
|
||||
# File(InputFormat) {#table_engines-file}
|
||||
|
||||
The data source is a file that stores data in one of the supported input formats (TabSeparated, Native, etc.).
|
||||
数据源是以 Clickhouse 支持的一种输入格式(TabSeparated,Native等)存储数据的文件。
|
||||
|
||||
Usage examples:
|
||||
用法示例:
|
||||
|
||||
- Data export from ClickHouse to file.
|
||||
- Convert data from one format to another.
|
||||
- Updating data in ClickHouse via editing a file on a disk.
|
||||
- 从 ClickHouse 导出数据到文件。
|
||||
- 将数据从一种格式转换为另一种格式。
|
||||
- 通过编辑磁盘上的文件来更新 ClickHouse 中的数据。
|
||||
|
||||
## Usage in ClickHouse Server
|
||||
## 在 ClickHouse 服务器中的使用
|
||||
|
||||
```
|
||||
File(Format)
|
||||
```
|
||||
|
||||
`Format` should be supported for either `INSERT` and `SELECT`. For the full list of supported formats see [Formats](../../interfaces/formats.md#formats).
|
||||
选用的 `Format` 需要支持 `INSERT` 或 `SELECT` 。有关支持格式的完整列表,请参阅 [格式](../../interfaces/formats.md#formats)。
|
||||
|
||||
ClickHouse does not allow to specify filesystem path for`File`. It will use folder defined by [path](../server_settings/settings.md) setting in server configuration.
|
||||
ClickHouse 不支持给 `File` 指定文件系统路径。它使用服务器配置中 [path](../server_settings/settings.md) 设定的文件夹。
|
||||
|
||||
When creating table using `File(Format)` it creates empty subdirectory in that folder. When data is written to that table, it's put into `data.Format` file in that subdirectory.
|
||||
使用 `File(Format)` 创建表时,它会在该文件夹中创建空的子目录。当数据写入该表时,它会写到该子目录中的 `data.Format` 文件中。
|
||||
|
||||
You may manually create this subfolder and file in server filesystem and then [ATTACH](../../query_language/misc.md) it to table information with matching name, so you can query data from that file.
|
||||
你也可以在服务器文件系统中手动创建这些子文件夹和文件,然后通过 [ATTACH](../../query_language/misc.md) 将其创建为具有对应名称的表,这样你就可以从该文件中查询数据了。
|
||||
|
||||
!!! warning
|
||||
Be careful with this funcionality, because ClickHouse does not keep track of external changes to such files. The result of simultaneous writes via ClickHouse and outside of ClickHouse is undefined.
|
||||
!!! 注意
|
||||
注意这个功能,因为 ClickHouse 不会跟踪这些文件在外部的更改。在 ClickHouse 中和 ClickHouse 外部同时写入会造成结果是不确定的。
|
||||
|
||||
**Example:**
|
||||
**示例:**
|
||||
|
||||
**1.** Set up the `file_engine_table` table:
|
||||
**1.** 创建 `file_engine_table` 表:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE file_engine_table (name String, value UInt32) ENGINE=File(TabSeparated)
|
||||
```
|
||||
|
||||
By default ClickHouse will create folder `/var/lib/clickhouse/data/default/file_engine_table`.
|
||||
默认情况下,Clickhouse 会创建目录 `/var/lib/clickhouse/data/default/file_engine_table` 。
|
||||
|
||||
**2.** Manually create `/var/lib/clickhouse/data/default/file_engine_table/data.TabSeparated` containing:
|
||||
**2.** 手动创建 `/var/lib/clickhouse/data/default/file_engine_table/data.TabSeparated` 文件,并且包含内容:
|
||||
|
||||
```bash
|
||||
$ cat data.TabSeparated
|
||||
@ -43,7 +43,7 @@ one 1
|
||||
two 2
|
||||
```
|
||||
|
||||
**3.** Query the data:
|
||||
**3.** 查询这些数据:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM file_engine_table
|
||||
@ -56,22 +56,22 @@ SELECT * FROM file_engine_table
|
||||
└──────┴───────┘
|
||||
```
|
||||
|
||||
## Usage in Clickhouse-local
|
||||
## 在 Clickhouse-local 中的使用
|
||||
|
||||
In [clickhouse-local](../utils/clickhouse-local.md) File engine accepts file path in addition to `Format`. Default input/output streams can be specified using numeric or human-readable names like `0` or `stdin`, `1` or `stdout`.
|
||||
**Example:**
|
||||
使用 [clickhouse-local](../utils/clickhouse-local.md) 时,File 引擎除了 `Format` 之外,还可以接受文件路径参数。可以使用数字或人类可读的名称来指定标准输入/输出流,例如 `0` 或 `stdin`,`1` 或 `stdout`。
|
||||
**例如:**
|
||||
|
||||
```bash
|
||||
$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table"
|
||||
```
|
||||
|
||||
## Details of Implementation
|
||||
## 功能实现
|
||||
|
||||
- Reads can be parallel, but not writes
|
||||
- Not supported:
|
||||
- 读操作可支持并发,但写操作不支持
|
||||
- 不支持:
|
||||
- `ALTER`
|
||||
- `SELECT ... SAMPLE`
|
||||
- Indices
|
||||
- Replication
|
||||
- 索引
|
||||
- 副本
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/file/) <!--hide-->
|
||||
[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/file/) <!--hide-->
|
||||
|
@ -192,7 +192,6 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data
|
||||
使用其他名称创建 MergeTree 表。将具有`ReplicatedMergeTree`表数据的目录中的所有数据移动到新表的数据目录中。然后删除`ReplicatedMergeTree`表并重新启动服务器。
|
||||
|
||||
如果你想在不启动服务器的情况下清除 `ReplicatedMergeTree` 表:
|
||||
If you want to get rid of a `ReplicatedMergeTree` table without launching the server:
|
||||
|
||||
- 删除元数据目录中的相应 `.sql` 文件(`/var/lib/clickhouse/metadata/`)。
|
||||
- 删除 ZooKeeper 中的相应路径(`/path_to_table/replica_name`)。
|
||||
|
@ -1,33 +1,33 @@
|
||||
# URL(URL, Format) {#table_engines-url}
|
||||
|
||||
Manages data on a remote HTTP/HTTPS server. This engine is similar
|
||||
to the [File](file.md) engine.
|
||||
用于管理远程 HTTP/HTTPS 服务器上的数据。该引擎类似
|
||||
[File](file.md) 引擎。
|
||||
|
||||
## Using the engine in the ClickHouse server
|
||||
## 在 ClickHouse 服务器中使用引擎
|
||||
|
||||
`The format` must be one that ClickHouse can use in
|
||||
`SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see
|
||||
[Formats](../../interfaces/formats.md#formats).
|
||||
`Format` 必须是 ClickHouse 可以用于
|
||||
`SELECT` 查询的一种格式,若有必要,还要可用于 `INSERT` 。有关支持格式的完整列表,请查看
|
||||
[Formats](../../interfaces/formats.md#formats)。
|
||||
|
||||
`The URL` must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server
|
||||
that uses HTTP or HTTPS. This does not require any
|
||||
additional headers for getting a response from the server.
|
||||
`URL` 必须符合统一资源定位符的结构。指定的URL必须指向一个
|
||||
HTTP 或 HTTPS 服务器。对于服务端响应,
|
||||
不需要任何额外的 HTTP 头标记。
|
||||
|
||||
`INSERT` and `SELECT` queries are transformed to `POST` and `GET` requests,
|
||||
respectively. For processing `POST` requests, the remote server must support
|
||||
[Chunked transfer encoding](https://en.wikipedia.org/wiki/Chunked_transfer_encoding).
|
||||
`INSERT` 和 `SELECT` 查询会分别转换为 `POST` 和 `GET` 请求。
|
||||
对于 `POST` 请求的处理,远程服务器必须支持
|
||||
[分块传输编码](https://en.wikipedia.org/wiki/Chunked_transfer_encoding)。
|
||||
|
||||
**Example:**
|
||||
**示例:**
|
||||
|
||||
**1.** Create a `url_engine_table` table on the server :
|
||||
**1.** 在 Clickhouse 服务上创建一个 `url_engine_table` 表:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE url_engine_table (word String, value UInt64)
|
||||
ENGINE=URL('http://127.0.0.1:12345/', CSV)
|
||||
```
|
||||
|
||||
**2.** Create a basic HTTP server using the standard Python 3 tools and
|
||||
start it:
|
||||
**2.** 用标准的 Python 3 工具库创建一个基本的 HTTP 服务并
|
||||
启动它:
|
||||
|
||||
```python3
|
||||
from http.server import BaseHTTPRequestHandler, HTTPServer
|
||||
@ -49,7 +49,7 @@ if __name__ == "__main__":
|
||||
python3 server.py
|
||||
```
|
||||
|
||||
**3.** Request data:
|
||||
**3.** 查询请求:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM url_engine_table
|
||||
@ -62,12 +62,12 @@ SELECT * FROM url_engine_table
|
||||
└───────┴───────┘
|
||||
```
|
||||
|
||||
## Details of Implementation
|
||||
## 功能实现
|
||||
|
||||
- Reads and writes can be parallel
|
||||
- Not supported:
|
||||
- `ALTER` and `SELECT...SAMPLE` operations.
|
||||
- Indexes.
|
||||
- Replication.
|
||||
- 读写操作都支持并发
|
||||
- 不支持:
|
||||
- `ALTER` 和 `SELECT...SAMPLE` 操作。
|
||||
- 索引。
|
||||
- 副本。
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/url/) <!--hide-->
|
||||
[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/url/) <!--hide-->
|
||||
|
Loading…
Reference in New Issue
Block a user