Skip to content

Commit

Permalink
[yugabyte#15767] DocDB: Revert vtables lib as it results in circular …
Browse files Browse the repository at this point in the history
…dependencies between master and vtables lib

Summary: This change reverts commit e8ce3d7 as this change resulted in circular dependencies between vtables and master lib. Circular dependency is only caught in `no-tcmalloc` build for which we don't allow any undefined symbols.

Test Plan: Jenkins

Reviewers: dfelsing, zdrudi, skedia, sergei, rthallam

Reviewed By: rthallam

Subscribers: sergei, skedia, rthallam, bogdan

Differential Revision: https://phabricator.dev.yugabyte.com/D22414
  • Loading branch information
karan-yb committed Jan 21, 2023
1 parent e525416 commit f9ca331
Show file tree
Hide file tree
Showing 49 changed files with 310 additions and 395 deletions.
1 change: 0 additions & 1 deletion CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -808,7 +808,6 @@ add_subdirectory(src/yb/rocksutil)
add_subdirectory(src/yb/docdb)
add_subdirectory(src/yb/yql)
add_subdirectory(src/yb/cdc)
add_subdirectory(src/yb/vtables)

include("${YB_SRC_ROOT}/ent/CMakeLists.txt")

Expand Down
39 changes: 29 additions & 10 deletions src/yb/master/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -115,30 +115,49 @@ set(MASTER_SRCS
tasks_tracker.cc
ts_descriptor.cc
ts_manager.cc
util/yql_vtable_helpers.cc
xcluster/xcluster_consumer_metrics.cc
xcluster/xcluster_safe_time_service.cc
yql_aggregates_vtable.cc
yql_auth_resource_role_permissions_index.cc
yql_auth_role_permissions_vtable.cc
yql_auth_roles_vtable.cc
yql_columns_vtable.cc
yql_empty_vtable.cc
yql_functions_vtable.cc
yql_indexes_vtable.cc
yql_keyspaces_vtable.cc
yql_local_vtable.cc
yql_partitions_vtable.cc
yql_peers_vtable.cc
yql_size_estimates_vtable.cc
yql_tables_vtable.cc
yql_triggers_vtable.cc
yql_types_vtable.cc
yql_views_vtable.cc
yql_virtual_table.cc
yql_vtable_iterator.cc
ysql_tablegroup_manager.cc
ysql_tablespace_manager.cc
ysql_transaction_ddl.cc
${MASTER_SRCS_EXTENSIONS})

set(MASTER_DEPS
${MASTER_PROTO_LIBS}
gutil
master_util
vtables
rpc_header_proto
yb_common
tablet
server_common
server_process
tablet
yrpc
gutil
yb_util
tserver
tserver_service_proto
${MASTER_PROTO_LIBS}
rpc_header_proto
master_util
version_info_proto
yb_common
yb_pggate
yb_pggate_flags
yb_util
yrpc
yb_pggate
)

ADD_YB_LIBRARY(master
Expand Down
71 changes: 43 additions & 28 deletions src/yb/master/catalog_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -109,8 +109,8 @@
#include "yb/gutil/walltime.h"

#include "yb/master/master_fwd.h"
#include "yb/master/async_rpc_tasks.h"
#include "yb/master/auto_flags_orchestrator.h"
#include "yb/master/async_rpc_tasks.h"
#include "yb/master/backfill_index.h"
#include "yb/master/catalog_entity_info.h"
#include "yb/master/catalog_loaders.h"
Expand All @@ -136,6 +136,23 @@
#include "yb/master/sys_catalog_constants.h"
#include "yb/master/ts_descriptor.h"
#include "yb/master/xcluster/xcluster_safe_time_service.h"
#include "yb/master/yql_aggregates_vtable.h"
#include "yb/master/yql_auth_resource_role_permissions_index.h"
#include "yb/master/yql_auth_role_permissions_vtable.h"
#include "yb/master/yql_auth_roles_vtable.h"
#include "yb/master/yql_columns_vtable.h"
#include "yb/master/yql_empty_vtable.h"
#include "yb/master/yql_functions_vtable.h"
#include "yb/master/yql_indexes_vtable.h"
#include "yb/master/yql_keyspaces_vtable.h"
#include "yb/master/yql_local_vtable.h"
#include "yb/master/yql_partitions_vtable.h"
#include "yb/master/yql_peers_vtable.h"
#include "yb/master/yql_size_estimates_vtable.h"
#include "yb/master/yql_tables_vtable.h"
#include "yb/master/yql_triggers_vtable.h"
#include "yb/master/yql_types_vtable.h"
#include "yb/master/yql_views_vtable.h"
#include "yb/master/ysql_tablegroup_manager.h"
#include "yb/master/ysql_transaction_ddl.h"

Expand Down Expand Up @@ -184,8 +201,6 @@
#include "yb/util/tsan_util.h"
#include "yb/util/uuid.h"

#include "yb/vtables/yql_all_vtables.h"

#include "yb/yql/pgwrapper/pg_wrapper.h"
#include "yb/yql/redis/redisserver/redis_constants.h"

Expand Down Expand Up @@ -1543,45 +1558,45 @@ Status CatalogManager::PrepareSystemTables(int64_t term) {
RETURN_NOT_OK(PrepareSysCatalogTable(term));

// Create the required system tables here.
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::PeersVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<PeersVTable>(
kSystemPeersTableName, kSystemNamespaceName, kSystemNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::LocalVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<LocalVTable>(
kSystemLocalTableName, kSystemNamespaceName, kSystemNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLKeyspacesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLKeyspacesVTable>(
kSystemSchemaKeyspacesTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId,
term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLTablesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLTablesVTable>(
kSystemSchemaTablesTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLColumnsVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLColumnsVTable>(
kSystemSchemaColumnsTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLSizeEstimatesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLSizeEstimatesVTable>(
kSystemSizeEstimatesTableName, kSystemNamespaceName, kSystemNamespaceId, term)));

// Empty tables.
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLAggregatesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLAggregatesVTable>(
kSystemSchemaAggregatesTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId,
term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLFunctionsVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLFunctionsVTable>(
kSystemSchemaFunctionsTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId,
term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLIndexesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLIndexesVTable>(
kSystemSchemaIndexesTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLTriggersVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLTriggersVTable>(
kSystemSchemaTriggersTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLViewsVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLViewsVTable>(
kSystemSchemaViewsTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::QLTypesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<QLTypesVTable>(
kSystemSchemaTypesTableName, kSystemSchemaNamespaceName, kSystemSchemaNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLPartitionsVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLPartitionsVTable>(
kSystemPartitionsTableName, kSystemNamespaceName, kSystemNamespaceId, term)));

// System auth tables.
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLAuthRolesVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLAuthRolesVTable>(
kSystemAuthRolesTableName, kSystemAuthNamespaceName, kSystemAuthNamespaceId, term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLAuthRolePermissionsVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLAuthRolePermissionsVTable>(
kSystemAuthRolePermissionsTableName, kSystemAuthNamespaceName, kSystemAuthNamespaceId,
term)));
RETURN_NOT_OK((PrepareSystemTableTemplate<yb::vtables::YQLAuthResourceRolePermissionsIndexVTable>(
RETURN_NOT_OK((PrepareSystemTableTemplate<YQLAuthResourceRolePermissionsIndexVTable>(
kSystemAuthResourceRolePermissionsIndexTableName, kSystemAuthNamespaceName,
kSystemAuthNamespaceId, term)));

Expand Down Expand Up @@ -1659,7 +1674,7 @@ Status CatalogManager::PrepareSystemTableTemplate(const TableName& table_name,
const NamespaceName& namespace_name,
const NamespaceId& namespace_id,
int64_t term) {
yb::vtables::YQLVirtualTable* vtable = new T(table_name, namespace_name, master_);
YQLVirtualTable* vtable = new T(table_name, namespace_name, master_);
return PrepareSystemTable(
table_name, namespace_name, namespace_id, vtable->schema(), term, vtable);
}
Expand All @@ -1669,8 +1684,8 @@ Status CatalogManager::PrepareSystemTable(const TableName& table_name,
const NamespaceId& namespace_id,
const Schema& schema,
int64_t term,
yb::vtables::YQLVirtualTable* vtable) {
std::unique_ptr<yb::vtables::YQLVirtualTable> yql_storage(vtable);
YQLVirtualTable* vtable) {
std::unique_ptr<YQLVirtualTable> yql_storage(vtable);

scoped_refptr<TableInfo> table = FindPtrOrNull(table_names_map_,
std::make_pair(namespace_id, table_name));
Expand Down Expand Up @@ -4831,7 +4846,7 @@ Result<bool> CatalogManager::IsCreateTableDone(const TableInfoPtr& table) {
if (DCHECK_IS_ON() &&
result &&
IsYcqlTable(*table) &&
yb::vtables::YQLPartitionsVTable::GeneratePartitionsVTableOnChanges() &&
YQLPartitionsVTable::GeneratePartitionsVTableOnChanges() &&
FLAGS_TEST_catalog_manager_check_yql_partitions_exist_for_is_create_table_done) {
Schema schema;
RETURN_NOT_OK(table->GetSchema(&schema));
Expand Down Expand Up @@ -12141,13 +12156,13 @@ Status CatalogManager::GetYQLPartitionsVTable(std::shared_ptr<SystemTablet>* tab
}

void CatalogManager::RebuildYQLSystemPartitions() {
if (yb::vtables::YQLPartitionsVTable::GeneratePartitionsVTableWithBgTask() ||
yb::vtables::YQLPartitionsVTable::GeneratePartitionsVTableOnChanges()) {
if (YQLPartitionsVTable::GeneratePartitionsVTableWithBgTask() ||
YQLPartitionsVTable::GeneratePartitionsVTableOnChanges()) {
SCOPED_LEADER_SHARED_LOCK(l, this);
if (l.IsInitializedAndIsLeader()) {
if (system_partitions_tablet_ != nullptr) {
Status s;
if (yb::vtables::YQLPartitionsVTable::GeneratePartitionsVTableWithBgTask()) {
if (YQLPartitionsVTable::GeneratePartitionsVTableWithBgTask()) {
// If we are not generating the vtable on changes, then we need to do a full refresh.
s = ResultToStatus(GetYqlPartitionsVtable().GenerateAndCacheData());
} else {
Expand Down Expand Up @@ -12279,8 +12294,8 @@ void CatalogManager::CheckTableDeleted(const TableInfoPtr& table) {
}), "Failed to submit update table task");
}

const yb::vtables::YQLPartitionsVTable& CatalogManager::GetYqlPartitionsVtable() const {
return down_cast<const yb::vtables::YQLPartitionsVTable&>(system_partitions_tablet_->QLStorage());
const YQLPartitionsVTable& CatalogManager::GetYqlPartitionsVtable() const {
return down_cast<const YQLPartitionsVTable&>(system_partitions_tablet_->QLStorage());
}

void CatalogManager::InitializeTableLoadState(
Expand Down
4 changes: 2 additions & 2 deletions src/yb/master/catalog_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -1114,7 +1114,7 @@ class CatalogManager : public tserver::TabletPeerLookupIf,
const NamespaceId& namespace_id,
const Schema& schema,
int64_t term,
yb::vtables::YQLVirtualTable* vtable) REQUIRES(mutex_);
YQLVirtualTable* vtable) REQUIRES(mutex_);

Status PrepareNamespace(YQLDatabase db_type,
const NamespaceName& name,
Expand Down Expand Up @@ -1983,7 +1983,7 @@ class CatalogManager : public tserver::TabletPeerLookupIf,

TSDescriptorVector GetAllLiveNotBlacklistedTServers() const;

const yb::vtables::YQLPartitionsVTable& GetYqlPartitionsVtable() const;
const YQLPartitionsVTable& GetYqlPartitionsVtable() const;

void InitializeTableLoadState(
const TableId& table_id, TSDescriptorVector ts_descs, CMPerTableLoadState* state);
Expand Down
9 changes: 2 additions & 7 deletions src/yb/master/master_fwd.h
Original file line number Diff line number Diff line change
Expand Up @@ -39,13 +39,6 @@ namespace yb {
class HostPort;
struct HostPortHash;

namespace vtables {

class YQLPartitionsVTable;
class YQLVirtualTable;

} // namespace vtables

namespace master {

class TSDescriptor;
Expand Down Expand Up @@ -87,6 +80,8 @@ class TSDescriptor;
class TSManager;
class UDTypeInfo;
class XClusterSafeTimeService;
class YQLPartitionsVTable;
class YQLVirtualTable;
class YsqlTablegroupManager;
class YsqlTablespaceManager;
class YsqlTransactionDdl;
Expand Down
17 changes: 8 additions & 9 deletions src/yb/master/system_tablet.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,19 @@
#include "yb/docdb/doc_read_context.h"

#include "yb/master/sys_catalog_constants.h"

#include "yb/vtables/yql_virtual_table.h"
#include "yb/master/yql_virtual_table.h"

namespace yb {
namespace master {

SystemTablet::SystemTablet(
const Schema& schema, std::unique_ptr<yb::vtables::YQLVirtualTable> yql_virtual_table,
const TabletId& tablet_id)
: log_prefix_(Format("T $0: ", tablet_id)), // Don't have UUID here to log in T XX P YY format.
doc_read_context_(
std::make_shared<docdb::DocReadContext>(log_prefix_, schema, kSysCatalogSchemaVersion)),
SystemTablet::SystemTablet(const Schema& schema, std::unique_ptr<YQLVirtualTable> yql_virtual_table,
const TabletId& tablet_id)
: log_prefix_(Format("T $0: ", tablet_id)), // Don't have UUID here to log in T XX P YY format.
doc_read_context_(std::make_shared<docdb::DocReadContext>(
log_prefix_, schema, kSysCatalogSchemaVersion)),
yql_virtual_table_(std::move(yql_virtual_table)),
tablet_id_(tablet_id) {}
tablet_id_(tablet_id) {
}

docdb::DocReadContextPtr SystemTablet::GetDocReadContext(const std::string& table_id) const {
// table_id is ignored. It should match the system table's id.
Expand Down
7 changes: 3 additions & 4 deletions src/yb/master/system_tablet.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,8 @@ namespace master {
// This is a virtual tablet that is used for our virtual tables in the system namespace.
class SystemTablet : public tablet::AbstractTablet {
public:
SystemTablet(
const Schema& schema, std::unique_ptr<yb::vtables::YQLVirtualTable> yql_virtual_table,
const TabletId& tablet_id);
SystemTablet(const Schema& schema, std::unique_ptr<YQLVirtualTable> yql_virtual_table,
const TabletId& tablet_id);

docdb::DocReadContextPtr GetDocReadContext(const std::string& table_id = "") const override;

Expand Down Expand Up @@ -100,7 +99,7 @@ class SystemTablet : public tablet::AbstractTablet {

const std::string log_prefix_;
docdb::DocReadContextPtr doc_read_context_;
std::unique_ptr<yb::vtables::YQLVirtualTable> yql_virtual_table_;
std::unique_ptr<YQLVirtualTable> yql_virtual_table_;
TabletId tablet_id_;
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
// under the License.
//

#include "yb/vtables/yql_vtable_helpers.h"
#include "yb/master/util/yql_vtable_helpers.h"

#include <future>

Expand All @@ -28,7 +28,7 @@
#include "yb/util/yb_partition.h"

namespace yb {
namespace vtables {
namespace master {
namespace util {

// Ideally, we want clients to use YB's own load-balancing policy for Cassandra to route the
Expand Down Expand Up @@ -63,7 +63,7 @@ bool RemoteEndpointMatchesList(const google::protobuf::RepeatedPtrField<HostPort
return false;
}

bool RemoteEndpointMatchesTServer(const yb::master::TSInformationPB& ts_info,
bool RemoteEndpointMatchesTServer(const TSInformationPB& ts_info,
const InetAddress& remote_endpoint) {
const auto& common = ts_info.registration().common();
if (RemoteEndpointMatchesList(common.private_rpc_addresses(), remote_endpoint)) {
Expand Down Expand Up @@ -95,7 +95,7 @@ QLValuePB GetReplicationValue(size_t replication_factor) {
}

PublicPrivateIPFutures GetPublicPrivateIPFutures(
const yb::master::TSInformationPB& ts_info, DnsResolver* resolver) {
const TSInformationPB& ts_info, DnsResolver* resolver) {
const auto& common = ts_info.registration().common();
PublicPrivateIPFutures result;

Expand Down Expand Up @@ -200,5 +200,5 @@ QLValuePB GetValueHelper<bool>::Apply(const bool bool_val, const DataType data_t
}

} // namespace util
} // namespace vtables
} // namespace master
} // namespace yb
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
#include "yb/util/uuid.h"

namespace yb {
namespace vtables {
namespace master {
namespace util {

template<class T> struct GetValueHelper;
Expand Down Expand Up @@ -78,17 +78,17 @@ QLValuePB GetTokensValue(size_t index, size_t node_count);

QLValuePB GetReplicationValue(size_t replication_factor);

bool RemoteEndpointMatchesTServer(
const yb::master::TSInformationPB& ts_info, const InetAddress& remote_endpoint);
bool RemoteEndpointMatchesTServer(const TSInformationPB& ts_info,
const InetAddress& remote_endpoint);

struct PublicPrivateIPFutures {
std::shared_future<Result<IpAddress>> private_ip_future;
std::shared_future<Result<IpAddress>> public_ip_future;
};

PublicPrivateIPFutures GetPublicPrivateIPFutures(
const master::TSInformationPB& ts_info, DnsResolver* resolver);
const TSInformationPB& ts_info, DnsResolver* resolver);

} // namespace util
} // namespace vtables
} // namespace master
} // namespace yb
Loading

0 comments on commit f9ca331

Please sign in to comment.