You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ga...@apache.org on 2019/04/01 11:54:58 UTC

[impala] 02/04: IMPALA-8312 : Alter database operations have race condition

This is an automated email from the ASF dual-hosted git repository.

gaborkaszab pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit b2a87797a8be076a8e57a91e8db2692ca693e2f3
Author: Vihang Karajgaonkar <vi...@cloudera.com>
AuthorDate: Thu Mar 14 16:56:26 2019 -0700

    IMPALA-8312 : Alter database operations have race condition
    
    This patch fixes a race condition in the alter database implementation
    in the catalogOpExecutor. The original implementation did a in-place
    modification of the metastore database object in the Db. This can lead
    to partially updated database object becoming visible to a reading
    thread causing potential problems. In order to fix the race, the
    patch makes a copy of the existing database object, modifies the copy
    and then atomically switches the actual database object with the
    modified copy. This is done while holding the metastoreddlLock, and
    then taking the write lock on the catalog version object which makes
    it consistent with the other catalog write operations.
    
    Added a test which consistently reproduces the race. The test creating
    many reader threads and a writer thread which continuously keeps
    changing the owner name and its type by issuing a alter database
    operation. The test fails without the patch. After the patch the test
    passes. The race also applies to the alter database set comment
    operation, although its hard to write a test for that code-path.
    
    Change-Id: I32c8c96a6029bf9d9db37ea8315f6c9603b5a2fc
    Reviewed-on: http://gerrit.cloudera.org:8080/12789
    Reviewed-by: Fredy Wijaya <fw...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../impala/catalog/CatalogServiceCatalog.java      |  29 ++-
 fe/src/main/java/org/apache/impala/catalog/Db.java |  35 ++-
 .../apache/impala/service/CatalogOpExecutor.java   |  50 ++---
 .../apache/impala/catalog/AlterDatabaseTest.java   | 241 +++++++++++++++++++++
 4 files changed, 314 insertions(+), 41 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 25b0b96..99689e4 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.impala.analysis.TableName;
@@ -972,6 +973,32 @@ public class CatalogServiceCatalog extends Catalog {
   }
 
   /**
+   * Updates the Db with the given metastore database object. Useful to doing in-place
+   * updates to the HMS db like in case of changing owner, adding comment or setting
+   * certain properties
+   * @param msDb The HMS database object to be used to update
+   * @return The updated Db object
+   * @throws DatabaseNotFoundException if Db with the name provided by given Database
+   * is not found in Catalog
+   */
+  public Db updateDb(Database msDb) throws DatabaseNotFoundException {
+    Preconditions.checkNotNull(msDb);
+    Preconditions.checkNotNull(msDb.getName());
+    versionLock_.writeLock().lock();
+    try {
+      Db db = getDb(msDb.getName());
+      if (db == null) {
+        throw new DatabaseNotFoundException("Database " + msDb.getName() + " not found");
+      }
+      db.setMetastoreDb(msDb.getName(), msDb);
+      db.setCatalogVersion(incrementAndGetCatalogVersion());
+      return db;
+    } finally {
+      versionLock_.writeLock().unlock();
+    }
+  }
+
+  /**
    * Adds a table in the topic update if its version is in the range
    * ('ctx.fromVersion', 'ctx.toVersion']. If the table's version is larger than
    * 'ctx.toVersion' and the table has skipped a topic update
@@ -1219,7 +1246,7 @@ public class CatalogServiceCatalog extends Catalog {
       // Contains native functions in it's params map.
       org.apache.hadoop.hive.metastore.api.Database msDb =
           msClient.getHiveClient().getDatabase(dbName);
-      tmpDb = new Db(dbName, null);
+      tmpDb = new Db(dbName, msDb);
       // Load native UDFs into the temporary db.
       loadFunctionsFromDbParams(tmpDb, msDb);
       // Load Java UDFs from HMS into the temporary db.
diff --git a/fe/src/main/java/org/apache/impala/catalog/Db.java b/fe/src/main/java/org/apache/impala/catalog/Db.java
index 5878b95..cc1a569 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Db.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Db.java
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.impala.analysis.ColumnDef;
 import org.apache.impala.analysis.KuduPartitionParam;
 import org.apache.impala.common.ImpalaException;
@@ -66,7 +68,10 @@ import com.google.common.collect.Lists;
  */
 public class Db extends CatalogObjectImpl implements FeDb {
   private static final Logger LOG = LoggerFactory.getLogger(Db.class);
-  private final TDatabase thriftDb_;
+  // TODO: We should have a consistent synchronization model for Db and Table
+  // Right now, we synchronize functions and thriftDb_ object in-place and do
+  // not take read lock on catalogVersion. See IMPALA-8366 for details
+  private final AtomicReference<TDatabase> thriftDb_ = new AtomicReference<>();
 
   public static final String FUNCTION_INDEX_PREFIX = "impala_registered_function_";
 
@@ -91,9 +96,8 @@ public class Db extends CatalogObjectImpl implements FeDb {
   private boolean isSystemDb_ = false;
 
   public Db(String name, org.apache.hadoop.hive.metastore.api.Database msDb) {
-    thriftDb_ = new TDatabase(name.toLowerCase());
-    thriftDb_.setMetastore_db(msDb);
-    tableCache_ = new CatalogObjectCache<Table>();
+    setMetastoreDb(name, msDb);
+    tableCache_ = new CatalogObjectCache<>();
     functions_ = new HashMap<>();
   }
 
@@ -110,7 +114,7 @@ public class Db extends CatalogObjectImpl implements FeDb {
    * Updates the hms parameters map by adding the input <k,v> pair.
    */
   private void putToHmsParameters(String k, String v) {
-    org.apache.hadoop.hive.metastore.api.Database msDb = thriftDb_.metastore_db;
+    org.apache.hadoop.hive.metastore.api.Database msDb = thriftDb_.get().metastore_db;
     Preconditions.checkNotNull(msDb);
     Map<String, String> hmsParams = msDb.getParameters();
     if (hmsParams == null) hmsParams = new HashMap<>();
@@ -124,7 +128,7 @@ public class Db extends CatalogObjectImpl implements FeDb {
    * corresponding to input k and it is removed, false otherwise.
    */
   private boolean removeFromHmsParameters(String k) {
-    org.apache.hadoop.hive.metastore.api.Database msDb = thriftDb_.metastore_db;
+    org.apache.hadoop.hive.metastore.api.Database msDb = thriftDb_.get().metastore_db;
     Preconditions.checkNotNull(msDb);
     if (msDb.getParameters() == null) return false;
     return msDb.getParameters().remove(k) != null;
@@ -133,9 +137,9 @@ public class Db extends CatalogObjectImpl implements FeDb {
   @Override // FeDb
   public boolean isSystemDb() { return isSystemDb_; }
   @Override // FeDb
-  public TDatabase toThrift() { return thriftDb_; }
+  public TDatabase toThrift() { return thriftDb_.get(); }
   @Override // FeDb
-  public String getName() { return thriftDb_.getDb_name(); }
+  public String getName() { return thriftDb_.get().getDb_name(); }
   @Override
   public TCatalogObjectType getCatalogObjectType() { return TCatalogObjectType.DATABASE; }
 
@@ -193,7 +197,7 @@ public class Db extends CatalogObjectImpl implements FeDb {
 
   @Override // FeDb
   public org.apache.hadoop.hive.metastore.api.Database getMetaStoreDb() {
-    return thriftDb_.getMetastore_db();
+    return thriftDb_.get().getMetastore_db();
   }
 
   @Override // FeDb
@@ -465,4 +469,17 @@ public class Db extends CatalogObjectImpl implements FeDb {
     }
     return resp;
   }
+
+  /**
+   * Replaces the metastore db object of this Db with the given Metastore Database object
+   * @param msDb
+   */
+  public void setMetastoreDb(String name, Database msDb) {
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(msDb);
+    // create the TDatabase first before atomically replacing setting it in the thriftDb_
+    TDatabase tDatabase = new TDatabase(name.toLowerCase());
+    tDatabase.setMetastore_db(msDb);
+    thriftDb_.set(tDatabase);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 705aa88..9b1e7ee 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -1212,7 +1212,7 @@ public class CatalogOpExecutor {
       } else {
         if (catalog_.addFunction(fn)) {
           // Flush DB changes to metastore
-          applyAlterDatabase(catalog_.getDb(fn.dbName()));
+          applyAlterDatabase(db.getMetaStoreDb());
           addedFunctions.add(fn.toTCatalogObject());
         }
       }
@@ -1745,7 +1745,7 @@ public class CatalogOpExecutor {
           }
         } else {
           // Flush DB changes to metastore
-          applyAlterDatabase(catalog_.getDb(fn.dbName()));
+          applyAlterDatabase(db.getMetaStoreDb());
           removedFunctions.add(fn.toTCatalogObject());
         }
       }
@@ -3122,10 +3122,10 @@ public class CatalogOpExecutor {
   /**
    * Updates the database object in the metastore.
    */
-  private void applyAlterDatabase(Db db)
+  private void applyAlterDatabase(Database msDb)
       throws ImpalaRuntimeException {
     try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-      msClient.getHiveClient().alterDatabase(db.getName(), db.getMetaStoreDb());
+      msClient.getHiveClient().alterDatabase(msDb.getName(), msDb);
     } catch (TException e) {
       throw new ImpalaRuntimeException(
           String.format(HMS_RPC_ERROR_FORMAT_STR, "alterDatabase"), e);
@@ -3753,17 +3753,16 @@ public class CatalogOpExecutor {
       throw new CatalogException("Database: " + dbName + " does not exist.");
     }
     synchronized (metastoreDdlLock_) {
-      Database msDb = db.getMetaStoreDb();
-      String originalComment = msDb.getDescription();
+      Database msDb = db.getMetaStoreDb().deepCopy();
       msDb.setDescription(comment);
       try {
-        applyAlterDatabase(db);
+        applyAlterDatabase(msDb);
       } catch (ImpalaRuntimeException e) {
-        msDb.setDescription(originalComment);
         throw e;
       }
+      Db updatedDb = catalog_.updateDb(msDb);
+      addDbToCatalogUpdate(updatedDb, response.result);
     }
-    addDbToCatalogUpdate(db, response.result);
     addSummary(response, "Updated database.");
   }
 
@@ -3788,45 +3787,34 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(params.owner_name);
     Preconditions.checkNotNull(params.owner_type);
     synchronized (metastoreDdlLock_) {
-      Database msDb = db.getMetaStoreDb();
+      Database msDb = db.getMetaStoreDb().deepCopy();
       String originalOwnerName = msDb.getOwnerName();
       PrincipalType originalOwnerType = msDb.getOwnerType();
       msDb.setOwnerName(params.owner_name);
       msDb.setOwnerType(PrincipalType.valueOf(params.owner_type.name()));
       try {
-        applyAlterDatabase(db);
+        applyAlterDatabase(msDb);
       } catch (ImpalaRuntimeException e) {
-        msDb.setOwnerName(originalOwnerName);
-        msDb.setOwnerType(originalOwnerType);
         throw e;
       }
       if (authzConfig_.isEnabled()) {
         authzManager_.updateDatabaseOwnerPrivilege(params.server_name, db.getName(),
-            originalOwnerName, originalOwnerType, db.getMetaStoreDb().getOwnerName(),
-            db.getMetaStoreDb().getOwnerType(), response);
+            originalOwnerName, originalOwnerType, msDb.getOwnerName(),
+            msDb.getOwnerType(), response);
       }
+      Db updatedDb = catalog_.updateDb(msDb);
+      addDbToCatalogUpdate(updatedDb, response.result);
     }
-    addDbToCatalogUpdate(db, response.result);
     addSummary(response, "Updated database.");
   }
 
   private void addDbToCatalogUpdate(Db db, TCatalogUpdateResult result) {
     Preconditions.checkNotNull(db);
-    // Updating the new catalog version and setting it to the DB catalog version while
-    // holding the catalog version lock for an atomic operation. Most DB operations are
-    // short-lived. It is unnecessary to have a fine-grained DB lock.
-    catalog_.getLock().writeLock().lock();
-    try {
-      long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
-      db.setCatalogVersion(newCatalogVersion);
-      TCatalogObject updatedCatalogObject = db.toTCatalogObject();
-      updatedCatalogObject.setCatalog_version(newCatalogVersion);
-      // TODO(todd): if client is a 'v2' impalad, only send back invalidation
-      result.addToUpdated_catalog_objects(updatedCatalogObject);
-      result.setVersion(updatedCatalogObject.getCatalog_version());
-    } finally {
-      catalog_.getLock().writeLock().unlock();
-    }
+    TCatalogObject updatedCatalogObject = db.toTCatalogObject();
+    updatedCatalogObject.setCatalog_version(updatedCatalogObject.getCatalog_version());
+    // TODO(todd): if client is a 'v2' impalad, only send back invalidation
+    result.addToUpdated_catalog_objects(updatedCatalogObject);
+    result.setVersion(updatedCatalogObject.getCatalog_version());
   }
 
   private void alterCommentOnTableOrView(TableName tableName, String comment,
diff --git a/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java b/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java
new file mode 100644
index 0000000..5313d22
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java
@@ -0,0 +1,241 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.catalog;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.impala.authorization.NoneAuthorizationFactory;
+import org.apache.impala.common.ImpalaException;
+import org.apache.impala.service.CatalogOpExecutor;
+import org.apache.impala.testutil.CatalogServiceTestCatalog;
+import org.apache.impala.testutil.ImpaladTestCatalog;
+import org.apache.impala.thrift.TAlterDbParams;
+import org.apache.impala.thrift.TAlterDbSetOwnerParams;
+import org.apache.impala.thrift.TAlterDbType;
+import org.apache.impala.thrift.TCreateDbParams;
+import org.apache.impala.thrift.TDdlExecRequest;
+import org.apache.impala.thrift.TDdlType;
+import org.apache.impala.thrift.TDropDbParams;
+import org.apache.impala.thrift.TOwnerType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test issues concurrent alter database operations to reproduce the race described in
+ * IMPALA-8312
+ */
+public class AlterDatabaseTest {
+  private static final String TEST_OWNER_1 = "user_1";
+  private static final PrincipalType TEST_TYPE_1 = PrincipalType.USER;
+
+  private static final String TEST_OWNER_2 = "user_2";
+  private static final PrincipalType TEST_TYPE_2 = PrincipalType.ROLE;
+
+  private static ImpaladTestCatalog catalog_;
+  private static CatalogOpExecutor catalogOpExecutor_;
+  private static final String TEST_ALTER_DB = "testAlterdb";
+  // number of reader threads which query the test database
+  private static final int NUM_READERS = 10;
+  // number of writer threads which change the database. We only need one currently
+  // since all the alterDatabase calls are serialized by metastoreDdlLock_ in
+  // CatalogOpExecutor
+  private static final int NUM_WRITERS = 1;
+
+  // barrier to make sure that readers and writers start at the same time
+  private final CyclicBarrier barrier_ =
+      new CyclicBarrier(NUM_READERS + NUM_WRITERS);
+  // toggle switch to change the database owner from user_1 to user_2 in each
+  // consecutive alter database call
+  private final AtomicBoolean toggler_ = new AtomicBoolean(false);
+
+  /**
+   * Sets up the test class by instantiating the catalog service
+   * @throws ImpalaException
+   */
+  @BeforeClass
+  public static void setUpTest() {
+    catalog_ = new ImpaladTestCatalog(CatalogServiceTestCatalog.create());
+    catalogOpExecutor_ =
+        new CatalogOpExecutor(catalog_.getSrcCatalog(), new NoneAuthorizationFactory());
+  }
+
+  /**
+   * Clean-up the database once the test completes
+   * @throws ImpalaException
+   */
+  @After
+  public void cleanUp() throws ImpalaException {
+    catalogOpExecutor_.execDdlRequest(dropDbRequest());
+  }
+
+  @Before
+  public void setUpDatabase() throws ImpalaException {
+    // cleanup and recreate any pre-existing testdb
+    catalogOpExecutor_.execDdlRequest(dropDbRequest());
+    catalogOpExecutor_.execDdlRequest(createDbRequest());
+    Db db = catalog_.getDb(TEST_ALTER_DB);
+    assertNotNull(db);
+    catalogOpExecutor_.execDdlRequest(getNextDdlRequest());
+    assertNotNull(catalog_.getDb(TEST_ALTER_DB));
+    String owner = db.getMetaStoreDb().getOwnerName();
+    assertTrue(owner.equals(TEST_OWNER_1) || owner.equals(TEST_OWNER_2));
+  }
+
+  /**
+   * Drops the test db from the test catalog
+   */
+  private static TDdlExecRequest dropDbRequest() {
+    TDdlExecRequest request = new TDdlExecRequest();
+    request.setDdl_type(TDdlType.DROP_DATABASE);
+    TDropDbParams dropDbParams = new TDropDbParams();
+    dropDbParams.setDb(TEST_ALTER_DB);
+    dropDbParams.setIf_exists(true);
+    dropDbParams.setCascade(true);
+    request.setDrop_db_params(dropDbParams);
+    return request;
+  }
+
+  /**
+   * Creates the test db in the catalog. Sets the owner to <code>TEST_OWNER_1</code>
+   */
+  private static TDdlExecRequest createDbRequest() {
+    TDdlExecRequest request = new TDdlExecRequest();
+    request.setDdl_type(TDdlType.CREATE_DATABASE);
+    TCreateDbParams createDbParams = new TCreateDbParams();
+    createDbParams.setDb(TEST_ALTER_DB);
+    createDbParams.setComment("test comment");
+    createDbParams.setOwner(TEST_OWNER_1);
+    request.setCreate_db_params(createDbParams);
+    return request;
+  }
+
+  /**
+   * Reader task to be used by the read threads. Calls into Catalog and validates if the
+   * owner and ownerType is valid
+   */
+  private class ValidateDbOwnerTask implements Callable<Void> {
+    @Override
+    public Void call() throws Exception {
+      barrier_.await();
+      for (int i = 0; i < 100; i++) {
+        Db testDb = catalog_.getDb(TEST_ALTER_DB);
+        validateOwner(testDb.getMetaStoreDb());
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Writer task to be used by the write threads. The task loops and issues many alter
+   * database set owner requests. Each call flips the owner and owner type from user_1
+   * (PrincipleType.USER) to user_2 (PrincipleType.ROLE) and vice versa.
+   */
+  private class SetOwnerTask implements Callable<Void> {
+    @Override
+    public Void call() throws Exception {
+      barrier_.await();
+      for (int i = 0; i < 100; i++) {
+        catalogOpExecutor_.execDdlRequest(getNextDdlRequest());
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Test creates multiple reader and writer threads which operate on the test database.
+   * The readers fetch the Db object and validate its owner information while the writer
+   * thread changes the owner of the database concurrently
+   */
+  @Test
+  public void testConcurrentAlterDbOps() throws Exception {
+    ExecutorService threadPool = Executors.newFixedThreadPool(NUM_READERS + NUM_WRITERS);
+    List<Future<Void>> results = new ArrayList<>(NUM_READERS + NUM_WRITERS);
+    for (int i = 0; i < NUM_WRITERS; i++) {
+      results.add(threadPool.submit(new SetOwnerTask()));
+    }
+    for (int i = 0; i < NUM_READERS; i++) {
+      results.add(threadPool.submit(new ValidateDbOwnerTask()));
+    }
+    try {
+      for (Future<Void> result : results) {
+        result.get(100, TimeUnit.SECONDS);
+      }
+    } finally {
+      threadPool.shutdownNow();
+    }
+  }
+
+  /**
+   * Creates ddl request to alter database set owner. Each invocation changes the owner
+   * from user_1 to user_2 and vice-versa.
+   */
+  private TDdlExecRequest getNextDdlRequest() {
+    TAlterDbSetOwnerParams alterDbSetOwnerParams = new TAlterDbSetOwnerParams();
+    if (toggler_.get()) {
+      alterDbSetOwnerParams.setOwner_name(TEST_OWNER_1);
+      alterDbSetOwnerParams.setOwner_type(TOwnerType.findByValue(0));
+      assertTrue(toggler_.compareAndSet(true, false));
+    } else {
+      alterDbSetOwnerParams.setOwner_name(TEST_OWNER_2);
+      alterDbSetOwnerParams.setOwner_type(TOwnerType.findByValue(1));
+      assertTrue(toggler_.compareAndSet(false, true));
+    }
+    TAlterDbParams alterDbParams = new TAlterDbParams();
+    alterDbParams.setDb(TEST_ALTER_DB);
+    alterDbParams.setAlter_type(TAlterDbType.SET_OWNER);
+    alterDbParams.setSet_owner_params(alterDbSetOwnerParams);
+    TDdlExecRequest request = new TDdlExecRequest();
+    request.setDdl_type(TDdlType.ALTER_DATABASE);
+    request.setAlter_db_params(alterDbParams);
+    return request;
+  }
+
+  /**
+   * Validates the owner information of the database. Makes sure that if the owner is
+   * user_1 its type is USER and if the owner is user_2 its type is ROLE
+   */
+  private void validateOwner(Database msDb) {
+    assertNotNull(msDb.getOwnerName());
+    assertNotNull(msDb.getOwnerType());
+    if (TEST_OWNER_1.equals(msDb.getOwnerName())) {
+      assertEquals("Owner " + TEST_OWNER_1 + " should have the type " + TEST_TYPE_1,
+          msDb.getOwnerType(), TEST_TYPE_1);
+    } else if (TEST_OWNER_2.equals(msDb.getOwnerName())) {
+      assertEquals("Owner " + TEST_OWNER_2 + " should have the type " + TEST_TYPE_2,
+          msDb.getOwnerType(), TEST_TYPE_2);
+    } else {
+      fail("Unknown owner for the database " + msDb.getOwnerName());
+    }
+  }
+}