Browse Source

YQ WM added unit tests for sls disable (#8687)

Pisarenko Grigoriy 6 months ago
parent
commit
085cd3d59c

+ 103 - 0
ydb/core/kqp/ut/scheme/kqp_scheme_ut.cpp

@@ -1,6 +1,7 @@
 #include <ydb/core/kqp/gateway/behaviour/resource_pool_classifier/fetcher.h>
 #include <ydb/core/kqp/ut/common/kqp_ut_common.h>
 #include <ydb/core/kqp/ut/common/columnshard.h>
+#include <ydb/core/kqp/workload_service/ut/common/kqp_workload_service_ut_common.h>
 #include <ydb/core/tx/columnshard/hooks/testing/controller.h>
 #include <ydb/core/tx/columnshard/test_helper/controllers.h>
 #include <ydb/core/formats/arrow/arrow_helpers.h>
@@ -6552,6 +6553,57 @@ Y_UNIT_TEST_SUITE(KqpScheme) {
             "Path does not exist");
     }
 
+    Y_UNIT_TEST(DisableResourcePoolsOnServerless) {
+        auto ydb = NWorkload::TYdbSetupSettings()
+            .CreateSampleTenants(true)
+            .EnableResourcePoolsOnServerless(false)
+            .Create();
+
+        auto checkDisabled = [](const auto& result) {
+            UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), NYdb::EStatus::GENERIC_ERROR, result.GetIssues().ToString());
+            UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), "Resource pools are disabled for serverless domains. Please contact your system administrator to enable it");
+        };
+
+        auto checkNotFound = [](const auto& result) {
+            UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), NYdb::EStatus::GENERIC_ERROR, result.GetIssues().ToString());
+            UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), "Path does not exist");
+        };
+
+        const auto& createSql = R"(
+            CREATE RESOURCE POOL MyResourcePool WITH (
+                CONCURRENT_QUERY_LIMIT=20,
+                QUEUE_SIZE=1000
+            );)";
+
+        const auto& alterSql = R"(
+            ALTER RESOURCE POOL MyResourcePool
+                SET (CONCURRENT_QUERY_LIMIT = 30, QUEUE_SIZE = 100),
+                RESET (QUERY_MEMORY_LIMIT_PERCENT_PER_NODE);
+            )";
+
+        const auto& dropSql = "DROP RESOURCE POOL MyResourcePool;";
+
+        auto settings = NWorkload::TQueryRunnerSettings().PoolId("");
+
+        // Dedicated, enabled
+        settings.Database(ydb->GetSettings().GetDedicatedTenantName()).NodeIndex(1);
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(createSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(alterSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(dropSql, settings));
+
+        // Shared, enabled
+        settings.Database(ydb->GetSettings().GetSharedTenantName()).NodeIndex(2);
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(createSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(alterSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(dropSql, settings));
+
+        // Serverless, disabled
+        settings.Database(ydb->GetSettings().GetServerlessTenantName()).NodeIndex(2);
+        checkDisabled(ydb->ExecuteQuery(createSql, settings));
+        checkNotFound(ydb->ExecuteQuery(alterSql, settings));
+        checkNotFound(ydb->ExecuteQuery(dropSql, settings));
+    }
+
     Y_UNIT_TEST(ResourcePoolsValidation) {
         NKikimrConfig::TAppConfig config;
         config.MutableFeatureFlags()->SetEnableResourcePools(true);
@@ -6821,6 +6873,57 @@ Y_UNIT_TEST_SUITE(KqpScheme) {
             "Classifier with name MyResourcePoolClassifier not found in database /Root");
     }
 
+    Y_UNIT_TEST(DisableResourcePoolClassifiersOnServerless) {
+        auto ydb = NWorkload::TYdbSetupSettings()
+            .CreateSampleTenants(true)
+            .EnableResourcePoolsOnServerless(false)
+            .Create();
+
+        auto checkDisabled = [](const auto& result) {
+            UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), NYdb::EStatus::GENERIC_ERROR, result.GetIssues().ToString());
+            UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), "Resource pool classifiers are disabled for serverless domains. Please contact your system administrator to enable it");
+        };
+
+        auto checkNotFound = [](const auto& result) {
+            UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), NYdb::EStatus::GENERIC_ERROR, result.GetIssues().ToString());
+            UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), "Classifier with name MyResourcePoolClassifier not found in database");
+        };
+
+        const auto& createSql = R"(
+            CREATE RESOURCE POOL CLASSIFIER MyResourcePoolClassifier WITH (
+                RANK=20,
+                RESOURCE_POOL="test_pool"
+            );)";
+
+        const auto& alterSql = R"(
+            ALTER RESOURCE POOL CLASSIFIER MyResourcePoolClassifier
+                SET (RANK = 1, MEMBERNAME = "test@user"),
+                RESET (RESOURCE_POOL);
+            )";
+
+        const auto& dropSql = "DROP RESOURCE POOL CLASSIFIER MyResourcePoolClassifier;";
+
+        auto settings = NWorkload::TQueryRunnerSettings().PoolId("");
+
+        // Dedicated, enabled
+        settings.Database(ydb->GetSettings().GetDedicatedTenantName()).NodeIndex(1);
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(createSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(alterSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(dropSql, settings));
+
+        // Shared, enabled
+        settings.Database(ydb->GetSettings().GetSharedTenantName()).NodeIndex(2);
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(createSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(alterSql, settings));
+        NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(dropSql, settings));
+
+        // Serverless, disabled
+        settings.Database(ydb->GetSettings().GetServerlessTenantName()).NodeIndex(2);
+        checkDisabled(ydb->ExecuteQuery(createSql, settings));
+        checkDisabled(ydb->ExecuteQuery(alterSql, settings));
+        checkNotFound(ydb->ExecuteQuery(dropSql, settings));
+    }
+
     Y_UNIT_TEST(ResourcePoolClassifiersValidation) {
         NKikimrConfig::TAppConfig config;
         config.MutableFeatureFlags()->SetEnableResourcePools(true);

+ 1 - 0
ydb/core/kqp/ut/scheme/ya.make

@@ -22,6 +22,7 @@ PEERDIR(
     library/cpp/threading/local_executor
     ydb/core/kqp
     ydb/core/kqp/ut/common
+    ydb/core/kqp/workload_service/ut/common
     ydb/core/tx/columnshard/hooks/testing
     ydb/library/yql/sql/pg
     ydb/library/yql/parser/pg_wrapper

+ 58 - 5
ydb/core/kqp/workload_service/ut/common/kqp_workload_service_ut_common.cpp

@@ -230,6 +230,7 @@ private:
     TAppConfig GetAppConfig() const {
         TAppConfig appConfig;
         appConfig.MutableFeatureFlags()->SetEnableResourcePools(Settings_.EnableResourcePools_);
+        appConfig.MutableFeatureFlags()->SetEnableResourcePoolsOnServerless(Settings_.EnableResourcePoolsOnServerless_);
         appConfig.MutableFeatureFlags()->SetEnableResourcePoolsCounters(true);
 
         return appConfig;
@@ -238,7 +239,7 @@ private:
     void SetLoggerSettings(TServerSettings& serverSettings) const {
         auto loggerInitializer = [](TTestActorRuntime& runtime) {
             runtime.SetLogPriority(NKikimrServices::KQP_WORKLOAD_SERVICE, NLog::EPriority::PRI_TRACE);
-            runtime.SetLogPriority(NKikimrServices::KQP_SESSION, NLog::EPriority::PRI_DEBUG);
+            runtime.SetLogPriority(NKikimrServices::KQP_SESSION, NLog::EPriority::PRI_TRACE);
         };
 
         serverSettings.SetLoggerInitializer(loggerInitializer);
@@ -255,16 +256,50 @@ private:
             .SetAppConfig(appConfig)
             .SetFeatureFlags(appConfig.GetFeatureFlags());
 
+        if (Settings_.CreateSampleTenants_) {
+            serverSettings
+                .SetDynamicNodeCount(2)
+                .AddStoragePoolType(Settings_.GetDedicatedTenantName())
+                .AddStoragePoolType(Settings_.GetSharedTenantName());
+        }
+
         SetLoggerSettings(serverSettings);
 
         return serverSettings;
     }
 
+    void SetupResourcesTenant(Ydb::Cms::CreateDatabaseRequest& request, Ydb::Cms::StorageUnits* storage, const TString& name) {
+        request.set_path(name);
+        storage->set_unit_kind(name);
+        storage->set_count(1);
+    }
+
+    void CreateTenants() {
+        {  // Dedicated
+            Ydb::Cms::CreateDatabaseRequest request;
+            SetupResourcesTenant(request, request.mutable_resources()->add_storage_units(), Settings_.GetDedicatedTenantName());
+            Tenants_->CreateTenant(std::move(request));
+        }
+
+        {  // Shared
+            Ydb::Cms::CreateDatabaseRequest request;
+            SetupResourcesTenant(request, request.mutable_shared_resources()->add_storage_units(), Settings_.GetSharedTenantName());
+            Tenants_->CreateTenant(std::move(request));
+        }
+
+        {  // Serverless
+            Ydb::Cms::CreateDatabaseRequest request;
+            request.set_path(Settings_.GetServerlessTenantName());
+            request.mutable_serverless_resources()->set_shared_database_path(Settings_.GetSharedTenantName());
+            Tenants_->CreateTenant(std::move(request));
+        }
+    }
+
     void InitializeServer() {
         ui32 grpcPort = PortManager_.GetPort();
         TServerSettings serverSettings = GetServerSettings(grpcPort);
 
-        Server_ = std::make_unique<TServer>(serverSettings);
+        Server_ = MakeIntrusive<TServer>(serverSettings);
         Server_->EnableGRpc(grpcPort);
         GetRuntime()->SetDispatchTimeout(FUTURE_WAIT_TIMEOUT);
 
@@ -277,10 +312,15 @@ private:
 
         TableClient_ = std::make_unique<NYdb::NTable::TTableClient>(*YdbDriver_, NYdb::NTable::TClientSettings().AuthToken("user@" BUILTIN_SYSTEM_DOMAIN));
         TableClientSession_ = std::make_unique<NYdb::NTable::TSession>(TableClient_->CreateSession().GetValueSync().GetSession());
+
+        Tenants_ = std::make_unique<TTenants>(Server_);
+        if (Settings_.CreateSampleTenants_) {
+            CreateTenants();
+        }
     }
 
     void CreateSamplePool() const {
-        if (!Settings_.EnableResourcePools_) {
+        if (!Settings_.EnableResourcePools_ || Settings_.CreateSampleTenants_) {
             return;
         }
 
@@ -483,7 +523,7 @@ private:
         request->SetQuery(query);
         request->SetType(NKikimrKqp::QUERY_TYPE_SQL_GENERIC_QUERY);
         request->SetAction(NKikimrKqp::QUERY_ACTION_EXECUTE);
-        request->SetDatabase(Settings_.DomainName_);
+        request->SetDatabase(settings.Database_ ? settings.Database_ : Settings_.DomainName_);
         request->SetPoolId(*settings.PoolId_);
 
         return event;
@@ -525,9 +565,10 @@ private:
     const TYdbSetupSettings Settings_;
 
     TPortManager PortManager_;
-    std::unique_ptr<TServer> Server_;
+    TServer::TPtr Server_;
     std::unique_ptr<TClient> Client_;
     std::unique_ptr<TDriver> YdbDriver_;
+    std::unique_ptr<TTenants> Tenants_;
 
     std::unique_ptr<NYdb::NTable::TTableClient> TableClient_;
     std::unique_ptr<NYdb::NTable::TSession> TableClientSession_;
@@ -586,6 +627,18 @@ TIntrusivePtr<IYdbSetup> TYdbSetupSettings::Create() const {
     return MakeIntrusive<TWorkloadServiceYdbSetup>(*this);
 }
 
+TString TYdbSetupSettings::GetDedicatedTenantName() const {
+    return TStringBuilder() << CanonizePath(DomainName_) << "/test-dedicated";
+}
+
+TString TYdbSetupSettings::GetSharedTenantName() const {
+    return TStringBuilder() << CanonizePath(DomainName_) << "/test-shared";
+}
+
+TString TYdbSetupSettings::GetServerlessTenantName() const {
+    return TStringBuilder() << CanonizePath(DomainName_) << "/test-serverless";
+}
+
 //// IYdbSetup
 
 void IYdbSetup::WaitFor(TDuration timeout, TString description, std::function<bool(TString&)> callback) {

+ 13 - 0
ydb/core/kqp/workload_service/ut/common/kqp_workload_service_ut_common.h

@@ -26,6 +26,7 @@ struct TQueryRunnerSettings {
     FLUENT_SETTING_DEFAULT(ui32, NodeIndex, 0);
     FLUENT_SETTING_DEFAULT(std::optional<TString>, PoolId, std::nullopt);
     FLUENT_SETTING_DEFAULT(TString, UserSID, "user@" BUILTIN_SYSTEM_DOMAIN);
+    FLUENT_SETTING_DEFAULT(TString, Database, "");
 
     // Runner settings
     FLUENT_SETTING_DEFAULT(bool, HangUpDuringExecution, false);
@@ -66,7 +67,9 @@ struct TYdbSetupSettings {
     // Cluster settings
     FLUENT_SETTING_DEFAULT(ui32, NodeCount, 1);
     FLUENT_SETTING_DEFAULT(TString, DomainName, "Root");
+    FLUENT_SETTING_DEFAULT(bool, CreateSampleTenants, false);
     FLUENT_SETTING_DEFAULT(bool, EnableResourcePools, true);
+    FLUENT_SETTING_DEFAULT(bool, EnableResourcePoolsOnServerless, false);
 
     // Default pool settings
     FLUENT_SETTING_DEFAULT(TString, PoolId, "sample_pool_id");
@@ -78,6 +81,10 @@ struct TYdbSetupSettings {
 
     NResourcePool::TPoolSettings GetDefaultPoolSettings() const;
     TIntrusivePtr<IYdbSetup> Create() const;
+
+    TString GetDedicatedTenantName() const;
+    TString GetSharedTenantName() const;
+    TString GetServerlessTenantName() const;
 };
 
 class IYdbSetup : public TThrRefBase {
@@ -127,6 +134,12 @@ struct TSampleQueries {
         UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), "Request timeout exceeded, cancelling after");
     }
 
+    template <typename TResult>
+    static void CheckNotFound(const TResult& result, const TString& poolId) {
+        UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), NYdb::EStatus::NOT_FOUND, result.GetIssues().ToString());
+        UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToString(), TStringBuilder() << "Resource pool " << poolId << " not found or you don't have access permissions");
+    }
+
     struct TSelect42 {
         static constexpr char Query[] = "SELECT 42;";
 

+ 51 - 0
ydb/core/kqp/workload_service/ut/kqp_workload_service_ut.cpp

@@ -1,4 +1,5 @@
 #include <ydb/core/base/appdata_fwd.h>
+#include <ydb/core/base/path.h>
 
 #include <ydb/core/kqp/workload_service/ut/common/kqp_workload_service_ut_common.h>
 
@@ -47,6 +48,56 @@ Y_UNIT_TEST_SUITE(KqpWorkloadService) {
         TSampleQueries::TSelect42::CheckResult(ydb->ExecuteQuery(TSampleQueries::TSelect42::Query, TQueryRunnerSettings().PoolId("another_pool_id")));
     }
 
+    Y_UNIT_TEST(WorkloadServiceDisabledByFeatureFlagOnServerless) {
+        auto ydb = TYdbSetupSettings()
+            .CreateSampleTenants(true)
+            .EnableResourcePoolsOnServerless(false)
+            .Create();
+
+        const TString& poolId = "another_pool_id";
+        auto settings = TQueryRunnerSettings().PoolId(poolId);
+
+        // Dedicated, enabled
+        TSampleQueries::CheckNotFound(ydb->ExecuteQuery(
+            TSampleQueries::TSelect42::Query,
+            settings.Database(ydb->GetSettings().GetDedicatedTenantName()).NodeIndex(1)
+        ), poolId);
+
+        // Shared, enabled
+        TSampleQueries::CheckNotFound(ydb->ExecuteQuery(
+            TSampleQueries::TSelect42::Query,
+            settings.Database(ydb->GetSettings().GetSharedTenantName()).NodeIndex(2)
+        ), poolId);
+
+        // Serverless, disabled
+        TSampleQueries::TSelect42::CheckResult(ydb->ExecuteQuery(
+            TSampleQueries::TSelect42::Query,
+            settings.Database(ydb->GetSettings().GetServerlessTenantName()).NodeIndex(2)
+        ));
+    }
+
+    Y_UNIT_TEST(WorkloadServiceDisabledByInvalidDatabasePath) {
+        auto ydb = TYdbSetupSettings().Create();
+
+        const TString& poolId = "another_pool_id";
+        auto settings = TQueryRunnerSettings().PoolId(poolId);
+
+        TSampleQueries::CheckNotFound(ydb->ExecuteQuery(TSampleQueries::TSelect42::Query, settings), poolId);
+
+        const TString& tabmleName = "sub_path";
+        ydb->ExecuteSchemeQuery(TStringBuilder() << R"(
+            CREATE TABLE )" << tabmleName << R"( (
+                Key Int32,
+                PRIMARY KEY (Key)
+            );
+        )");
+
+        TSampleQueries::TSelect42::CheckResult(ydb->ExecuteQuery(
+            TSampleQueries::TSelect42::Query,
+            settings.Database(TStringBuilder() << CanonizePath(ydb->GetSettings().DomainName_) << "/" << tabmleName)
+        ));
+    }
+
     TQueryRunnerResultAsync StartQueueSizeCheckRequests(TIntrusivePtr<IYdbSetup> ydb, const TQueryRunnerSettings& settings) {
         // One of these requests should be rejected by QueueSize
         auto firstRequest = ydb->ExecuteQueryAsync(TSampleQueries::TSelect42::Query, settings);

+ 44 - 0
ydb/core/testlib/test_client.cpp

@@ -2719,6 +2719,50 @@ namespace Tests {
         return Server->DynamicNodes();
     }
 
+    void TTenants::CreateTenant(Ydb::Cms::CreateDatabaseRequest request, ui32 nodes, TDuration timeout) {
+        const TString path = request.path();
+        const bool serverless = request.has_serverless_resources();
+
+        // Create new tenant
+        auto& runtime = *Server->GetRuntime();
+        const auto result = NKikimr::NRpcService::DoLocalRpc<NKikimr::NGRpcService::TGrpcRequestOperationCall<Ydb::Cms::CreateDatabaseRequest, Ydb::Cms::CreateDatabaseResponse>>(
+            std::move(request), "", "", runtime.GetActorSystem(0), true
+        ).ExtractValueSync();
+
+        if (result.operation().status() != Ydb::StatusIds::SUCCESS) {
+            NYql::TIssues issues;
+            NYql::IssuesFromMessage(result.operation().issues(), issues);
+            ythrow yexception() << "Failed to create tenant " << path << ", " << result.operation().status() << ", reason:\n" << issues.ToString();
+        }
+
+        // Run new tenant
+        if (!serverless) {
+            Run(path, nodes);
+        }
+
+        // Wait tenant is up
+        Ydb::Cms::GetDatabaseStatusResult getTenantResult;
+        const TActorId edgeActor = runtime.AllocateEdgeActor();
+        const TInstant start = TInstant::Now();
+        while (TInstant::Now() - start <= timeout) {
+            auto getTenantRequest = std::make_unique<NConsole::TEvConsole::TEvGetTenantStatusRequest>();
+            getTenantRequest->Record.MutableRequest()->set_path(path);
+            runtime.SendToPipe(MakeConsoleID(), edgeActor, getTenantRequest.release(), 0, GetPipeConfigWithRetries());
+
+            auto response = runtime.GrabEdgeEvent<NConsole::TEvConsole::TEvGetTenantStatusResponse>(edgeActor, timeout);
+            if (!response) {
+                ythrow yexception() << "Waiting CMS get tenant response timeout. Last tenant description:\n" << getTenantResult.DebugString();
+            }
+            response->Get()->Record.GetResponse().operation().result().UnpackTo(&getTenantResult);
+            if (getTenantResult.state() == Ydb::Cms::GetDatabaseStatusResult::RUNNING) {
+                return;
+            }
+
+            Sleep(TDuration::MilliSeconds(100));
+        }
+        ythrow yexception() << "Waiting tenant status RUNNING timeout. Spent time " << TInstant::Now() - start << " exceeds limit " << timeout << ". Last tenant description:\n" << getTenantResult.DebugString();
+    }
+
     TVector<ui32> &TTenants::Nodes(const TString &name) {
         return Tenants[name];
     }

+ 2 - 0
ydb/core/testlib/test_client.h

@@ -635,6 +635,8 @@ namespace Tests {
         ui32 Availabe() const;
         ui32 Capacity() const;
 
+        void CreateTenant(Ydb::Cms::CreateDatabaseRequest request, ui32 nodes = 1, TDuration timeout = TDuration::Seconds(30));
+
     private:
         TVector<ui32>& Nodes(const TString &name);
         void StopNode(const TString /*name*/, ui32 nodeIdx);

+ 1 - 1
ydb/tests/tools/kqprun/kqprun.cpp

@@ -354,7 +354,7 @@ protected:
                 TablesMapping[tableName] = filePath;
             });
 
-        options.AddLongOption('c', "app-config", "File with app config (TAppConfig for ydb tennant)")
+        options.AddLongOption('c', "app-config", "File with app config (TAppConfig for ydb tenant)")
             .RequiredArgument("file")
             .DefaultValue("./configuration/app_config.conf")
             .Handler1([this](const NLastGetopt::TOptsParser* option) {