Rename table

This commit is contained in:
alesapin 2020-10-27 16:07:50 +03:00
parent e40c509392
commit 3d3d246bdc
7 changed files with 12 additions and 11 deletions

View File

@ -52,6 +52,7 @@ std::string getEndpointId(const std::string & node_id)
return "DataPartsExchange:" + node_id;
}
/// Simple functor for tracking fetch progress in system.fetches table.
struct ReplicatedFetchReadCallback
{
ReplicatedFetchList::Entry & replicated_fetch_entry;

View File

@ -1,4 +1,4 @@
#include <Storages/System/StorageSystemFetches.h>
#include <Storages/System/StorageSystemReplicatedFetches.h>
#include <Storages/MergeTree/ReplicatedFetchList.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
@ -8,7 +8,7 @@
namespace DB
{
NamesAndTypesList StorageSystemFetches::getNamesAndTypes()
NamesAndTypesList StorageSystemReplicatedFetches::getNamesAndTypes()
{
return {
{"database", std::make_shared<DataTypeString>()},
@ -30,7 +30,7 @@ NamesAndTypesList StorageSystemFetches::getNamesAndTypes()
};
}
void StorageSystemFetches::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
void StorageSystemReplicatedFetches::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
const auto access = context.getAccess();
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES);

View File

@ -11,11 +11,11 @@ namespace DB
class Context;
/// system.fetches table. Takes data from context.getReplicatedFetchList()
class StorageSystemFetches final : public ext::shared_ptr_helper<StorageSystemFetches>, public IStorageSystemOneBlock<StorageSystemFetches >
class StorageSystemReplicatedFetches final : public ext::shared_ptr_helper<StorageSystemReplicatedFetches>, public IStorageSystemOneBlock<StorageSystemReplicatedFetches >
{
friend struct ext::shared_ptr_helper<StorageSystemFetches>;
friend struct ext::shared_ptr_helper<StorageSystemReplicatedFetches>;
public:
std::string getName() const override { return "SystemFetches"; }
std::string getName() const override { return "SystemReplicatedFetches"; }
static NamesAndTypesList getNamesAndTypes();

View File

@ -19,7 +19,7 @@
#include <Storages/System/StorageSystemMacros.h>
#include <Storages/System/StorageSystemMerges.h>
#include <Storages/System/StorageSystemFetches.h>
#include <Storages/System/StorageSystemReplicatedFetches.h>
#include <Storages/System/StorageSystemMetrics.h>
#include <Storages/System/StorageSystemModels.h>
#include <Storages/System/StorageSystemMutations.h>
@ -138,7 +138,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
attach<StorageSystemClusters>(system_database, "clusters");
attach<StorageSystemGraphite>(system_database, "graphite_retentions");
attach<StorageSystemMacros>(system_database, "macros");
attach<StorageSystemFetches>(system_database, "fetches");
attach<StorageSystemReplicatedFetches>(system_database, "replicated_fetches");
if (has_zookeeper)
attach<StorageSystemZooKeeper>(system_database, "zookeeper");

View File

@ -156,7 +156,6 @@ SRCS(
System/StorageSystemDistributionQueue.cpp
System/StorageSystemEnabledRoles.cpp
System/StorageSystemEvents.cpp
System/StorageSystemFetches.cpp
System/StorageSystemFormats.cpp
System/StorageSystemFunctions.cpp
System/StorageSystemGrants.cpp
@ -179,6 +178,7 @@ SRCS(
System/StorageSystemQuotasUsage.cpp
System/StorageSystemQuotaUsage.cpp
System/StorageSystemReplicas.cpp
System/StorageSystemReplicatedFetches.cpp
System/StorageSystemReplicationQueue.cpp
System/StorageSystemRoleGrants.cpp
System/StorageSystemRoles.cpp

View File

@ -27,7 +27,7 @@ def started_cluster():
def get_random_string(length):
return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length))
def test_system_fetches(started_cluster):
def test_system_replicated_fetches(started_cluster):
node1.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '1') ORDER BY tuple()")
node2.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '2') ORDER BY tuple()")
@ -38,7 +38,7 @@ def test_system_fetches(started_cluster):
node2.query("SYSTEM START FETCHES t")
fetches_result = []
for _ in range(1000):
result = json.loads(node2.query("SELECT * FROM system.fetches FORMAT JSON"))
result = json.loads(node2.query("SELECT * FROM system.replicated_fetches FORMAT JSON"))
if not result["data"]:
if fetches_result:
break