mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Rename table
This commit is contained in:
parent
e40c509392
commit
3d3d246bdc
@ -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;
|
||||
|
@ -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);
|
@ -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();
|
||||
|
@ -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");
|
||||
|
@ -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
|
||||
|
@ -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
|
Loading…
Reference in New Issue
Block a user