diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ParentControllerRegionState.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ParentControllerRegionState.java index 6ec23bfc130..d7f95ed875d 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ParentControllerRegionState.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ParentControllerRegionState.java @@ -2,8 +2,8 @@ /** * Enum representing the state of the region where the parent controller resides. - * (i.e., Region dc-0 is ACTIVE while Region dc-1 is PASSIVE - * This means that ParentController in dc-0 is serving requests while ParentController in dc-1 is rejecting requests) + * i.e., Region dc-0 is ACTIVE while Region dc-1 is PASSIVE + * This means that ParentController in dc-0 is serving requests while ParentController in dc-1 is rejecting requests */ public enum ParentControllerRegionState { /** The region is active, so the parent controller in the region is serving requests */ diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/AdminSparkServer.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/AdminSparkServer.java index 3765eb45577..aaa7e31ce5e 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/AdminSparkServer.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/AdminSparkServer.java @@ -307,150 +307,344 @@ public boolean startInner() throws Exception { return writeMenu("Set Active Version", SET_VERSION.getPath(), SET_VERSION.getParams()); }); - httpService.get(LIST_CHILD_CLUSTERS.getPath(), controllerRoutes.getChildControllers(admin)); - httpService.get(LIST_STORES.getPath(), storesRoutes.getAllStores(admin)); - httpService.get(CLUSTER_HEALTH_STORES.getPath(), storesRoutes.getAllStoresStatuses(admin)); - httpService.get(STORE.getPath(), storesRoutes.getStore(admin)); - httpService.get(FUTURE_VERSION.getPath(), storesRoutes.getFutureVersion(admin)); - httpService.get(BACKUP_VERSION.getPath(), storesRoutes.getBackupVersion(admin)); - httpService.post(SET_TOPIC_COMPACTION.getPath(), storesRoutes.setTopicCompaction(admin)); - - httpService.post(UPDATE_CLUSTER_CONFIG.getPath(), clusterRoutes.updateClusterConfig(admin)); - httpService.post(WIPE_CLUSTER.getPath(), clusterRoutes.wipeCluster(admin)); - httpService.post(REPLICATE_META_DATA.getPath(), newClusterBuildOutRoutes.copyOverStoreSchemasAndConfigs(admin)); - - httpService.get(JOB.getPath(), jobRoutes.jobStatus(admin)); - httpService.post(KILL_OFFLINE_PUSH_JOB.getPath(), jobRoutes.killOfflinePushJob(admin)); - httpService.post(SKIP_ADMIN.getPath(), skipAdminRoute.skipAdminMessage(admin)); - - httpService.post(EMPTY_PUSH.getPath(), createVersion.emptyPush(admin)); - httpService.post(END_OF_PUSH.getPath(), createVersion.writeEndOfPush(admin)); - httpService.post(REQUEST_TOPIC.getPath(), createVersion.requestTopicForPushing(admin)); - httpService.post(ADD_VERSION.getPath(), createVersion.addVersionAndStartIngestion(admin)); - httpService.post(NEW_STORE.getPath(), createStoreRoute.createStore(admin)); + httpService.get( + LIST_CHILD_CLUSTERS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.getChildControllers(admin))); + httpService.get( + LIST_STORES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getAllStores(admin))); + httpService.get( + CLUSTER_HEALTH_STORES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getAllStoresStatuses(admin))); + httpService.get(STORE.getPath(), new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getStore(admin))); + httpService.get( + FUTURE_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getFutureVersion(admin))); + httpService.get( + BACKUP_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getBackupVersion(admin))); + httpService.post( + SET_TOPIC_COMPACTION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.setTopicCompaction(admin))); + + httpService.post( + UPDATE_CLUSTER_CONFIG.getPath(), + new VeniceParentControllerRegionStateHandler(admin, clusterRoutes.updateClusterConfig(admin))); + httpService.post( + WIPE_CLUSTER.getPath(), + new VeniceParentControllerRegionStateHandler(admin, clusterRoutes.wipeCluster(admin))); + httpService.post( + REPLICATE_META_DATA.getPath(), + new VeniceParentControllerRegionStateHandler( + admin, + newClusterBuildOutRoutes.copyOverStoreSchemasAndConfigs(admin))); + + httpService.get(JOB.getPath(), new VeniceParentControllerRegionStateHandler(admin, jobRoutes.jobStatus(admin))); + httpService.post( + KILL_OFFLINE_PUSH_JOB.getPath(), + new VeniceParentControllerRegionStateHandler(admin, jobRoutes.killOfflinePushJob(admin))); + httpService.post( + SKIP_ADMIN.getPath(), + new VeniceParentControllerRegionStateHandler(admin, skipAdminRoute.skipAdminMessage(admin))); + + httpService.post( + EMPTY_PUSH.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createVersion.emptyPush(admin))); + httpService.post( + END_OF_PUSH.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createVersion.writeEndOfPush(admin))); + httpService.post( + REQUEST_TOPIC.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createVersion.requestTopicForPushing(admin))); + httpService.post( + ADD_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createVersion.addVersionAndStartIngestion(admin))); + httpService.post( + NEW_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createStoreRoute.createStore(admin))); httpService.get( CHECK_RESOURCE_CLEANUP_FOR_STORE_CREATION.getPath(), - createStoreRoute.checkResourceCleanupForStoreCreation(admin)); - httpService.post(DELETE_STORE.getPath(), storesRoutes.deleteStore(admin)); - httpService.post(UPDATE_STORE.getPath(), storesRoutes.updateStore(admin)); - - httpService.get(STORE_MIGRATION_ALLOWED.getPath(), clusterRoutes.isStoreMigrationAllowed(admin)); - httpService.post(MIGRATE_STORE.getPath(), storesRoutes.migrateStore(admin)); - httpService.post(COMPLETE_MIGRATION.getPath(), storesRoutes.completeMigration(admin)); - httpService.post(ABORT_MIGRATION.getPath(), storesRoutes.abortMigration(admin)); - - httpService.post(ENABLE_STORE.getPath(), storesRoutes.enableStore(admin)); - httpService.post(DELETE_ALL_VERSIONS.getPath(), storesRoutes.deleteAllVersions(admin)); - httpService.post(DELETE_OLD_VERSION.getPath(), storesRoutes.deleteOldVersions(admin)); - httpService.post(SET_VERSION.getPath(), storesRoutes.setCurrentVersion(admin)); - httpService.post(ROLLBACK_TO_BACKUP_VERSION.getPath(), storesRoutes.rollbackToBackupVersion(admin)); - httpService.post(ROLL_FORWARD_TO_FUTURE_VERSION.getPath(), storesRoutes.rollForwardToFutureVersion(admin)); - httpService - .post(SEND_HEARTBEAT_TIMESTAMP_TO_SYSTEM_STORE.getPath(), storesRoutes.sendHeartbeatToSystemStore(admin)); - httpService - .get(GET_HEARTBEAT_TIMESTAMP_FROM_SYSTEM_STORE.getPath(), storesRoutes.getHeartbeatFromSystemStore(admin)); - httpService.get(ClUSTER_HEALTH_INSTANCES.getPath(), nodesAndReplicas.listAllNodesStatus(admin)); - httpService.get(LIST_NODES.getPath(), nodesAndReplicas.listAllNodes(admin)); - httpService.get(LIST_REPLICAS.getPath(), nodesAndReplicas.listReplicasForStore(admin)); - httpService.get(NODE_REPLICAS.getPath(), nodesAndReplicas.listReplicasForStorageNode(admin)); - httpService.get(NODE_REMOVABLE.getPath(), nodesAndReplicas.isNodeRemovable(admin)); - httpService.get(NODE_REPLICAS_READINESS.getPath(), nodesAndReplicas.nodeReplicasReadiness(admin)); - httpService.post(ALLOW_LIST_ADD_NODE.getPath(), nodesAndReplicas.addNodeIntoAllowList(admin)); - httpService.post(ALLOW_LIST_REMOVE_NODE.getPath(), nodesAndReplicas.removeNodeFromAllowList(admin)); - httpService.post(REMOVE_NODE.getPath(), nodesAndReplicas.removeNodeFromCluster(admin)); + new VeniceParentControllerRegionStateHandler( + admin, + createStoreRoute.checkResourceCleanupForStoreCreation(admin))); + httpService.post( + DELETE_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.deleteStore(admin))); + httpService.post( + UPDATE_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.updateStore(admin))); + + httpService.get( + STORE_MIGRATION_ALLOWED.getPath(), + new VeniceParentControllerRegionStateHandler(admin, clusterRoutes.isStoreMigrationAllowed(admin))); + httpService.post( + MIGRATE_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.migrateStore(admin))); + httpService.post( + COMPLETE_MIGRATION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.completeMigration(admin))); + httpService.post( + ABORT_MIGRATION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.abortMigration(admin))); + + httpService.post( + ENABLE_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.enableStore(admin))); + httpService.post( + DELETE_ALL_VERSIONS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.deleteAllVersions(admin))); + httpService.post( + DELETE_OLD_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.deleteOldVersions(admin))); + httpService.post( + SET_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.setCurrentVersion(admin))); + httpService.post( + ROLLBACK_TO_BACKUP_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.rollbackToBackupVersion(admin))); + httpService.post( + ROLL_FORWARD_TO_FUTURE_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.rollForwardToFutureVersion(admin))); + httpService.post( + SEND_HEARTBEAT_TIMESTAMP_TO_SYSTEM_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.sendHeartbeatToSystemStore(admin))); + httpService.get( + GET_HEARTBEAT_TIMESTAMP_FROM_SYSTEM_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getHeartbeatFromSystemStore(admin))); + httpService.get( + ClUSTER_HEALTH_INSTANCES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.listAllNodesStatus(admin))); + httpService.get( + LIST_NODES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.listAllNodes(admin))); + httpService.get( + LIST_REPLICAS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.listReplicasForStore(admin))); + httpService.get( + NODE_REPLICAS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.listReplicasForStorageNode(admin))); + httpService.get( + NODE_REMOVABLE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.isNodeRemovable(admin))); + httpService.get( + NODE_REPLICAS_READINESS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.nodeReplicasReadiness(admin))); + httpService.post( + ALLOW_LIST_ADD_NODE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.addNodeIntoAllowList(admin))); + httpService.post( + ALLOW_LIST_REMOVE_NODE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.removeNodeFromAllowList(admin))); + httpService.post( + REMOVE_NODE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, nodesAndReplicas.removeNodeFromCluster(admin))); // Operations for key schema/value schema - httpService.get(GET_KEY_SCHEMA.getPath(), schemaRoutes.getKeySchema(admin)); - httpService.post(ADD_VALUE_SCHEMA.getPath(), schemaRoutes.addValueSchema(admin)); - httpService.post(ADD_DERIVED_SCHEMA.getPath(), schemaRoutes.addDerivedSchema(admin)); - httpService.get(GET_VALUE_SCHEMA.getPath(), schemaRoutes.getValueSchema(admin)); - httpService.post(GET_VALUE_SCHEMA_ID.getPath(), schemaRoutes.getValueSchemaID(admin)); - httpService.post(GET_VALUE_OR_DERIVED_SCHEMA_ID.getPath(), schemaRoutes.getValueOrDerivedSchemaID(admin)); - httpService.get(GET_ALL_VALUE_SCHEMA.getPath(), schemaRoutes.getAllValueSchema(admin)); - httpService.get(GET_ALL_VALUE_AND_DERIVED_SCHEMA.getPath(), schemaRoutes.getAllValueAndDerivedSchema(admin)); - httpService.post(REMOVE_DERIVED_SCHEMA.getPath(), schemaRoutes.removeDerivedSchema(admin)); - httpService - .get(GET_ALL_REPLICATION_METADATA_SCHEMAS.getPath(), schemaRoutes.getAllReplicationMetadataSchemas(admin)); + httpService.get( + GET_KEY_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getKeySchema(admin))); + httpService.post( + ADD_VALUE_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.addValueSchema(admin))); + httpService.post( + ADD_DERIVED_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.addDerivedSchema(admin))); + httpService.get( + GET_VALUE_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getValueSchema(admin))); + httpService.post( + GET_VALUE_SCHEMA_ID.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getValueSchemaID(admin))); + httpService.post( + GET_VALUE_OR_DERIVED_SCHEMA_ID.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getValueOrDerivedSchemaID(admin))); + httpService.get( + GET_ALL_VALUE_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getAllValueSchema(admin))); + httpService.get( + GET_ALL_VALUE_AND_DERIVED_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getAllValueAndDerivedSchema(admin))); + httpService.post( + REMOVE_DERIVED_SCHEMA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.removeDerivedSchema(admin))); + httpService.get( + GET_ALL_REPLICATION_METADATA_SCHEMAS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, schemaRoutes.getAllReplicationMetadataSchemas(admin))); - httpService.post(SET_OWNER.getPath(), storesRoutes.setOwner(admin)); - httpService.post(SET_PARTITION_COUNT.getPath(), storesRoutes.setPartitionCount(admin)); + httpService + .post(SET_OWNER.getPath(), new VeniceParentControllerRegionStateHandler(admin, storesRoutes.setOwner(admin))); + httpService.post( + SET_PARTITION_COUNT.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.setPartitionCount(admin))); - httpService.get(MASTER_CONTROLLER.getPath(), controllerRoutes.getLeaderController(admin)); + httpService.get( + MASTER_CONTROLLER.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.getLeaderController(admin))); // This API should be used by CORP controller only. VPJ could talk to any of controllers in CORP to find who is the // current leader CORP controller. In other colos, router will find the leader controller instead of calling this // API. - httpService.get(LEADER_CONTROLLER.getPath(), controllerRoutes.getLeaderController(admin)); + httpService.get( + LEADER_CONTROLLER.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.getLeaderController(admin))); - httpService.get(EXECUTION.getPath(), adminCommandExecutionRoutes.getExecution(admin)); - httpService.get(LAST_SUCCEED_EXECUTION_ID.getPath(), adminCommandExecutionRoutes.getLastSucceedExecutionId(admin)); + httpService.get( + EXECUTION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, adminCommandExecutionRoutes.getExecution(admin))); + httpService.get( + LAST_SUCCEED_EXECUTION_ID.getPath(), + new VeniceParentControllerRegionStateHandler( + admin, + adminCommandExecutionRoutes.getLastSucceedExecutionId(admin))); - httpService.get(STORAGE_ENGINE_OVERHEAD_RATIO.getPath(), storesRoutes.getStorageEngineOverheadRatio(admin)); + httpService.get( + STORAGE_ENGINE_OVERHEAD_RATIO.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getStorageEngineOverheadRatio(admin))); - httpService.post(ENABLE_THROTTLING.getPath(), routersClusterConfigRoutes.enableThrottling(admin)); - httpService - .post(ENABLE_MAX_CAPACITY_PROTECTION.getPath(), routersClusterConfigRoutes.enableMaxCapacityProtection(admin)); - httpService.post(ENABLE_QUOTA_REBALANCED.getPath(), routersClusterConfigRoutes.enableQuotaRebalanced(admin)); + httpService.post( + ENABLE_THROTTLING.getPath(), + new VeniceParentControllerRegionStateHandler(admin, routersClusterConfigRoutes.enableThrottling(admin))); + httpService.post( + ENABLE_MAX_CAPACITY_PROTECTION.getPath(), + new VeniceParentControllerRegionStateHandler( + admin, + routersClusterConfigRoutes.enableMaxCapacityProtection(admin))); + httpService.post( + ENABLE_QUOTA_REBALANCED.getPath(), + new VeniceParentControllerRegionStateHandler(admin, routersClusterConfigRoutes.enableQuotaRebalanced(admin))); - httpService.get(GET_ROUTERS_CLUSTER_CONFIG.getPath(), routersClusterConfigRoutes.getRoutersClusterConfig(admin)); + httpService.get( + GET_ROUTERS_CLUSTER_CONFIG.getPath(), + new VeniceParentControllerRegionStateHandler(admin, routersClusterConfigRoutes.getRoutersClusterConfig(admin))); - httpService.get(GET_ALL_MIGRATION_PUSH_STRATEGIES.getPath(), migrationRoutes.getAllMigrationPushStrategies(admin)); - httpService.get(SET_MIGRATION_PUSH_STRATEGY.getPath(), migrationRoutes.setMigrationPushStrategy(admin)); + httpService.get( + GET_ALL_MIGRATION_PUSH_STRATEGIES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, migrationRoutes.getAllMigrationPushStrategies(admin))); + httpService.get( + SET_MIGRATION_PUSH_STRATEGY.getPath(), + new VeniceParentControllerRegionStateHandler(admin, migrationRoutes.setMigrationPushStrategy(admin))); - httpService.get(CLUSTER_DISCOVERY.getPath(), ClusterDiscovery.discoverCluster(admin)); - httpService.get(LIST_BOOTSTRAPPING_VERSIONS.getPath(), versionRoute.listBootstrappingVersions(admin)); + httpService.get( + CLUSTER_DISCOVERY.getPath(), + new VeniceParentControllerRegionStateHandler(admin, ClusterDiscovery.discoverCluster(admin))); + httpService.get( + LIST_BOOTSTRAPPING_VERSIONS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, versionRoute.listBootstrappingVersions(admin))); - httpService.post(OFFLINE_PUSH_INFO.getPath(), createVersion.uploadPushInfo(admin)); + httpService.post( + OFFLINE_PUSH_INFO.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createVersion.uploadPushInfo(admin))); - httpService.post(UPLOAD_PUSH_JOB_STATUS.getPath(), jobRoutes.uploadPushJobStatus(admin)); - httpService.post(SEND_PUSH_JOB_DETAILS.getPath(), jobRoutes.sendPushJobDetails(admin)); + httpService.post( + UPLOAD_PUSH_JOB_STATUS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, jobRoutes.uploadPushJobStatus(admin))); + httpService.post( + SEND_PUSH_JOB_DETAILS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, jobRoutes.sendPushJobDetails(admin))); httpService.post( CONFIGURE_ACTIVE_ACTIVE_REPLICATION_FOR_CLUSTER.getPath(), - storesRoutes.enableActiveActiveReplicationForCluster(admin)); - httpService.post(UPDATE_ACL.getPath(), createStoreRoute.updateAclForStore(admin)); - httpService.get(GET_ACL.getPath(), createStoreRoute.getAclForStore(admin)); - httpService.get(DELETE_ACL.getPath(), createStoreRoute.deleteAclForStore(admin)); - httpService.get(GET_DELETABLE_STORE_TOPICS.getPath(), storesRoutes.getDeletableStoreTopics(admin)); - httpService - .get(GET_ONGOING_INCREMENTAL_PUSH_VERSIONS.getPath(), jobRoutes.getOngoingIncrementalPushVersions(admin)); - httpService.get(GET_REPUSH_INFO.getPath(), storesRoutes.getRepushInfo(admin)); - httpService.get(COMPARE_STORE.getPath(), storesRoutes.compareStore(admin)); - httpService.get(GET_STALE_STORES_IN_CLUSTER.getPath(), storesRoutes.getStaleStoresInCluster(admin)); - httpService.get(GET_STORES_IN_CLUSTER.getPath(), storesRoutes.getStoresInCluster(admin)); - httpService.get(GET_STORE_LARGEST_USED_VERSION.getPath(), storesRoutes.getStoreLargestUsedVersion(admin)); - httpService.get(GET_REGION_PUSH_DETAILS.getPath(), storesRoutes.getRegionPushDetails(admin)); - httpService.get(LIST_STORE_PUSH_INFO.getPath(), storesRoutes.listStorePushInfo(admin)); + new VeniceParentControllerRegionStateHandler( + admin, + storesRoutes.enableActiveActiveReplicationForCluster(admin))); + httpService.post( + UPDATE_ACL.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createStoreRoute.updateAclForStore(admin))); + httpService.get( + GET_ACL.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createStoreRoute.getAclForStore(admin))); + httpService.get( + DELETE_ACL.getPath(), + new VeniceParentControllerRegionStateHandler(admin, createStoreRoute.deleteAclForStore(admin))); + httpService.get( + GET_DELETABLE_STORE_TOPICS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getDeletableStoreTopics(admin))); + httpService.get( + GET_ONGOING_INCREMENTAL_PUSH_VERSIONS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, jobRoutes.getOngoingIncrementalPushVersions(admin))); + httpService.get( + GET_REPUSH_INFO.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getRepushInfo(admin))); + httpService.get( + COMPARE_STORE.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.compareStore(admin))); + httpService.get( + GET_STALE_STORES_IN_CLUSTER.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getStaleStoresInCluster(admin))); + httpService.get( + GET_STORES_IN_CLUSTER.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getStoresInCluster(admin))); + httpService.get( + GET_STORE_LARGEST_USED_VERSION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getStoreLargestUsedVersion(admin))); + httpService.get( + GET_REGION_PUSH_DETAILS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getRegionPushDetails(admin))); + httpService.get( + LIST_STORE_PUSH_INFO.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.listStorePushInfo(admin))); - httpService.post(PREPARE_DATA_RECOVERY.getPath(), dataRecoveryRoutes.prepareDataRecovery(admin)); + httpService.post( + PREPARE_DATA_RECOVERY.getPath(), + new VeniceParentControllerRegionStateHandler(admin, dataRecoveryRoutes.prepareDataRecovery(admin))); httpService.get( IS_STORE_VERSION_READY_FOR_DATA_RECOVERY.getPath(), - dataRecoveryRoutes.isStoreVersionReadyForDataRecovery(admin)); - httpService.post(DATA_RECOVERY.getPath(), dataRecoveryRoutes.dataRecovery(admin)); - httpService.get(GET_KAFKA_TOPIC_CONFIGS.getPath(), controllerRoutes.getKafkaTopicConfigs(admin)); - httpService - .post(UPDATE_KAFKA_TOPIC_LOG_COMPACTION.getPath(), controllerRoutes.updateKafkaTopicLogCompaction(admin)); - httpService.post(UPDATE_KAFKA_TOPIC_RETENTION.getPath(), controllerRoutes.updateKafkaTopicRetention(admin)); + new VeniceParentControllerRegionStateHandler( + admin, + dataRecoveryRoutes.isStoreVersionReadyForDataRecovery(admin))); + httpService.post( + DATA_RECOVERY.getPath(), + new VeniceParentControllerRegionStateHandler(admin, dataRecoveryRoutes.dataRecovery(admin))); + httpService.get( + GET_KAFKA_TOPIC_CONFIGS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.getKafkaTopicConfigs(admin))); + httpService.post( + UPDATE_KAFKA_TOPIC_LOG_COMPACTION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.updateKafkaTopicLogCompaction(admin))); + httpService.post( + UPDATE_KAFKA_TOPIC_RETENTION.getPath(), + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.updateKafkaTopicRetention(admin))); httpService.post( UPDATE_KAFKA_TOPIC_MIN_IN_SYNC_REPLICA.getPath(), - controllerRoutes.updateKafkaTopicMinInSyncReplica(admin)); + new VeniceParentControllerRegionStateHandler(admin, controllerRoutes.updateKafkaTopicMinInSyncReplica(admin))); - httpService.get(GET_ADMIN_TOPIC_METADATA.getPath(), adminTopicMetadataRoutes.getAdminTopicMetadata(admin)); - httpService.post(UPDATE_ADMIN_TOPIC_METADATA.getPath(), adminTopicMetadataRoutes.updateAdminTopicMetadata(admin)); + httpService.get( + GET_ADMIN_TOPIC_METADATA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, adminTopicMetadataRoutes.getAdminTopicMetadata(admin))); + httpService.post( + UPDATE_ADMIN_TOPIC_METADATA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, adminTopicMetadataRoutes.updateAdminTopicMetadata(admin))); - httpService.post(DELETE_KAFKA_TOPIC.getPath(), storesRoutes.deleteKafkaTopic(admin)); - httpService.post(REMOVE_STORE_FROM_GRAVEYARD.getPath(), storesRoutes.removeStoreFromGraveyard(admin)); + httpService.post( + DELETE_KAFKA_TOPIC.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.deleteKafkaTopic(admin))); + httpService.post( + REMOVE_STORE_FROM_GRAVEYARD.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.removeStoreFromGraveyard(admin))); - httpService.post(CREATE_STORAGE_PERSONA.getPath(), storagePersonaRoutes.createStoragePersona(admin)); - httpService.get(GET_STORAGE_PERSONA.getPath(), storagePersonaRoutes.getStoragePersona(admin)); - httpService.post(DELETE_STORAGE_PERSONA.getPath(), storagePersonaRoutes.deleteStoragePersona(admin)); - httpService.post(UPDATE_STORAGE_PERSONA.getPath(), storagePersonaRoutes.updateStoragePersona(admin)); + httpService.post( + CREATE_STORAGE_PERSONA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.createStoragePersona(admin))); + httpService.get( + GET_STORAGE_PERSONA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.getStoragePersona(admin))); + httpService.post( + DELETE_STORAGE_PERSONA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.deleteStoragePersona(admin))); + httpService.post( + UPDATE_STORAGE_PERSONA.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.updateStoragePersona(admin))); httpService.get( GET_STORAGE_PERSONA_ASSOCIATED_WITH_STORE.getPath(), - storagePersonaRoutes.getPersonaAssociatedWithStore(admin)); - httpService.get(GET_CLUSTER_STORAGE_PERSONAS.getPath(), storagePersonaRoutes.getClusterStoragePersonas(admin)); - httpService.get(GET_INUSE_SCHEMA_IDS.getPath(), storesRoutes.getInUseSchemaIds(admin)); + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.getPersonaAssociatedWithStore(admin))); + httpService.get( + GET_CLUSTER_STORAGE_PERSONAS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storagePersonaRoutes.getClusterStoragePersonas(admin))); + httpService.get( + GET_INUSE_SCHEMA_IDS.getPath(), + new VeniceParentControllerRegionStateHandler(admin, storesRoutes.getInUseSchemaIds(admin))); - httpService - .post(CLEANUP_INSTANCE_CUSTOMIZED_STATES.getPath(), clusterRoutes.cleanupInstanceCustomizedStates(admin)); + httpService.post( + CLEANUP_INSTANCE_CUSTOMIZED_STATES.getPath(), + new VeniceParentControllerRegionStateHandler(admin, clusterRoutes.cleanupInstanceCustomizedStates(admin))); httpService.awaitInitialization(); // Wait for server to be initialized Exception e = initFailure.get(); diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java index 22ccaf796f9..ef47461b916 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java @@ -2,7 +2,6 @@ import static com.linkedin.venice.HttpConstants.HTTP_GET; import static com.linkedin.venice.VeniceConstants.CONTROLLER_SSL_CERTIFICATE_ATTRIBUTE_NAME; -import static com.linkedin.venice.controller.ParentControllerRegionState.ACTIVE; import static com.linkedin.venice.controller.server.CreateVersion.overrideSourceRegionAddressForIncrementalPushJob; import static com.linkedin.venice.controllerapi.ControllerApiConstants.CLUSTER; import static com.linkedin.venice.controllerapi.ControllerApiConstants.HOSTNAME; @@ -252,7 +251,6 @@ public void testRequestTopicForIncPushCanUseEmergencyRegionWhenItIsSet() throws doReturn("dc-0").when(admin).getRegionName(); doReturn(true).when(admin).whetherEnableBatchPushFromAdmin(STORE_NAME); doReturn(true).when(admin).isParent(); - doReturn(ACTIVE).when(admin).getParentControllerRegionState(); doReturn(true).when(admin).isActiveActiveReplicationEnabledInAllRegion(any(), any(), anyBoolean()); doReturn(store).when(admin).getStore(CLUSTER_NAME, STORE_NAME); doReturn("default-src.region.io").when(admin).getKafkaBootstrapServers(anyBoolean());