You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2019/07/05 08:10:17 UTC

[hive] branch master updated: HIVE-21918: Handle each Alter Database types in a separate desc / operation (Miklos Gergely via Jesus Camacho Rodriguez)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9a9978f  HIVE-21918: Handle each Alter Database types in a separate desc / operation (Miklos Gergely via Jesus Camacho Rodriguez)
9a9978f is described below

commit 9a9978f18a172e388baa7988cd4ef4926d6dd847
Author: Miklos Gergely <mg...@hortonworks.com>
AuthorDate: Fri Jul 5 10:09:21 2019 +0200

    HIVE-21918: Handle each Alter Database types in a separate desc / operation (Miklos Gergely via Jesus Camacho Rodriguez)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../ql/ddl/database/AbstractAlterDatabaseDesc.java |  54 ++++++++++
 ...on.java => AbstractAlterDatabaseOperation.java} |  60 +----------
 .../hive/ql/ddl/database/AlterDatabaseDesc.java    | 110 ---------------------
 .../ddl/database/AlterDatabaseSetLocationDesc.java |  42 ++++++++
 .../AlterDatabaseSetLocationOperation.java         |  57 +++++++++++
 .../ql/ddl/database/AlterDatabaseSetOwnerDesc.java |  44 +++++++++
 .../database/AlterDatabaseSetOwnerOperation.java   |  39 ++++++++
 .../database/AlterDatabaseSetPropertiesDesc.java   |  46 +++++++++
 .../AlterDatabaseSetPropertiesOperation.java       |  48 +++++++++
 .../ql/exec/repl/bootstrap/load/LoadDatabase.java  |   9 +-
 .../incremental/IncrementalLoadTasksBuilder.java   |   5 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  |  13 ++-
 .../repl/load/message/AlterDatabaseHandler.java    |  12 ++-
 .../repl/load/message/CreateDatabaseHandler.java   |   9 +-
 .../test/queries/clientpositive/alter_db_owner.q   |   1 +
 .../results/clientpositive/alter_db_owner.q.out    |  17 ++++
 .../results/clientpositive/db_ddl_explain.q.out    |   2 +-
 17 files changed, 381 insertions(+), 187 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseDesc.java
new file mode 100644
index 0000000..20374ef
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseDesc.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.ql.ddl.DDLDesc;
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for ALTER DATABASE commands.
+ */
+public abstract class AbstractAlterDatabaseDesc implements DDLDesc, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String databaseName;
+  private final ReplicationSpec replicationSpec;
+
+  public AbstractAlterDatabaseDesc(String databaseName, ReplicationSpec replicationSpec) {
+    this.databaseName = databaseName;
+    this.replicationSpec = replicationSpec;
+  }
+
+  @Explain(displayName="name", explainLevels = {Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  /**
+   * @return what kind of replication scope this alter is running under.
+   * This can result in a "ALTER IF NEWER THAN" kind of semantic
+   */
+  public ReplicationSpec getReplicationSpec() {
+    return this.replicationSpec;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseOperation.java
similarity index 50%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseOperation.java
index 9ce3b62..61076fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AbstractAlterDatabaseOperation.java
@@ -18,11 +18,8 @@
 
 package org.apache.hadoop.hive.ql.ddl.database;
 
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.Map;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
@@ -32,8 +29,8 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 /**
  * Operation process of altering a database.
  */
-public class AlterDatabaseOperation extends DDLOperation<AlterDatabaseDesc> {
-  public AlterDatabaseOperation(DDLOperationContext context, AlterDatabaseDesc desc) {
+public abstract class AbstractAlterDatabaseOperation<T extends AbstractAlterDatabaseDesc> extends DDLOperation<T> {
+  public AbstractAlterDatabaseOperation(DDLOperationContext context, T desc) {
     super(context, desc);
   }
 
@@ -52,60 +49,11 @@ public class AlterDatabaseOperation extends DDLOperation<AlterDatabaseDesc> {
       return 0; // no replacement, the existing database state is newer than our update.
     }
 
-    switch (desc.getAlterType()) {
-    case ALTER_PROPERTY:
-      alterProperties(database, params);
-      break;
-
-    case ALTER_OWNER:
-      alterOwner(database);
-      break;
-
-    case ALTER_LOCATION:
-      alterLocation(database);
-      break;
-
-    default:
-      throw new AssertionError("Unsupported alter database type! : " + desc.getAlterType());
-    }
+    doAlteration(database, params);
 
     context.getDb().alterDatabase(database.getName(), database);
     return 0;
   }
 
-  private void alterProperties(Database database, Map<String, String> params) {
-    Map<String, String> newParams = desc.getDatabaseProperties();
-
-    // if both old and new params are not null, merge them
-    if (params != null && newParams != null) {
-      params.putAll(newParams);
-      database.setParameters(params);
-    } else {
-      // if one of them is null, replace the old params with the new one
-      database.setParameters(newParams);
-    }
-  }
-
-  private void alterOwner(Database database) {
-    database.setOwnerName(desc.getOwnerPrincipal().getName());
-    database.setOwnerType(desc.getOwnerPrincipal().getType());
-  }
-
-  private void alterLocation(Database database) throws HiveException {
-    try {
-      String newLocation = desc.getLocation();
-      URI locationURI = new URI(newLocation);
-      if (!locationURI.isAbsolute() || StringUtils.isBlank(locationURI.getScheme())) {
-        throw new HiveException(ErrorMsg.BAD_LOCATION_VALUE, newLocation);
-      }
-
-      if (newLocation.equals(database.getLocationUri())) {
-        LOG.info("AlterDatabase skipped. No change in location.");
-      } else {
-        database.setLocationUri(newLocation);
-      }
-    } catch (URISyntaxException e) {
-      throw new HiveException(e);
-    }
-  }
+  protected abstract void doAlteration(Database database, Map<String, String> params) throws HiveException;
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseDesc.java
deleted file mode 100644
index f83c74c..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseDesc.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.hadoop.hive.ql.ddl.database;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.ddl.DDLDesc;
-import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
-import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
-import org.apache.hadoop.hive.ql.plan.Explain;
-import org.apache.hadoop.hive.ql.plan.Explain.Level;
-
-/**
- * DDL task description for ALTER DATABASE commands.
- */
-@Explain(displayName = "Alter Database", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-public class AlterDatabaseDesc implements DDLDesc, Serializable {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * Supported type of alter db commands.
-   * Only altering the database property and owner is currently supported
-   */
-  public enum AlterDbType {
-    ALTER_PROPERTY, ALTER_OWNER, ALTER_LOCATION
-  };
-
-  private final AlterDbType alterType;
-  private final String databaseName;
-  private final Map<String, String> dbProperties;
-  private final ReplicationSpec replicationSpec;
-  private final PrincipalDesc ownerPrincipal;
-  private final String location;
-
-  public AlterDatabaseDesc(String databaseName, Map<String, String> dbProperties, ReplicationSpec replicationSpec) {
-    this.alterType = AlterDbType.ALTER_PROPERTY;
-    this.databaseName = databaseName;
-    this.dbProperties = dbProperties;
-    this.replicationSpec = replicationSpec;
-    this.ownerPrincipal = null;
-    this.location = null;
-  }
-
-  public AlterDatabaseDesc(String databaseName, PrincipalDesc ownerPrincipal, ReplicationSpec replicationSpec) {
-    this.alterType = AlterDbType.ALTER_OWNER;
-    this.databaseName = databaseName;
-    this.dbProperties = null;
-    this.replicationSpec = replicationSpec;
-    this.ownerPrincipal = ownerPrincipal;
-    this.location = null;
-  }
-
-  public AlterDatabaseDesc(String databaseName, String location) {
-    this.alterType = AlterDbType.ALTER_LOCATION;
-    this.databaseName = databaseName;
-    this.dbProperties = null;
-    this.replicationSpec = null;
-    this.ownerPrincipal = null;
-    this.location = location;
-  }
-
-  public AlterDbType getAlterType() {
-    return alterType;
-  }
-
-  @Explain(displayName="name", explainLevels = {Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  @Explain(displayName="properties")
-  public Map<String, String> getDatabaseProperties() {
-    return dbProperties;
-  }
-
-  /**
-   * @return what kind of replication scope this alter is running under.
-   * This can result in a "ALTER IF NEWER THAN" kind of semantic
-   */
-  public ReplicationSpec getReplicationSpec() {
-    return this.replicationSpec;
-  }
-
-  @Explain(displayName="owner")
-  public PrincipalDesc getOwnerPrincipal() {
-    return ownerPrincipal;
-  }
-
-  @Explain(displayName="location")
-  public String getLocation() {
-    return location;
-  }
-}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationDesc.java
new file mode 100644
index 0000000..cb7fb3d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationDesc.java
@@ -0,0 +1,42 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for ALTER DATABASE ... SET LOCATION ... commands.
+ */
+@Explain(displayName = "Set Database Location", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class AlterDatabaseSetLocationDesc extends AbstractAlterDatabaseDesc {
+  private static final long serialVersionUID = 1L;
+
+  private final String location;
+
+  public AlterDatabaseSetLocationDesc(String databaseName, String location) {
+    super(databaseName, null);
+    this.location = location;
+  }
+
+  @Explain(displayName="location")
+  public String getLocation() {
+    return location;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationOperation.java
new file mode 100644
index 0000000..e136cdb
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetLocationOperation.java
@@ -0,0 +1,57 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Operation process of altering a database's location.
+ */
+public class AlterDatabaseSetLocationOperation extends AbstractAlterDatabaseOperation<AlterDatabaseSetLocationDesc> {
+  public AlterDatabaseSetLocationOperation(DDLOperationContext context, AlterDatabaseSetLocationDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  protected void doAlteration(Database database, Map<String, String> params) throws HiveException {
+    try {
+      String newLocation = desc.getLocation();
+      URI locationURI = new URI(newLocation);
+      if (!locationURI.isAbsolute() || StringUtils.isBlank(locationURI.getScheme())) {
+        throw new HiveException(ErrorMsg.BAD_LOCATION_VALUE, newLocation);
+      }
+
+      if (newLocation.equals(database.getLocationUri())) {
+        LOG.info("AlterDatabase skipped. No change in location.");
+      } else {
+        database.setLocationUri(newLocation);
+      }
+    } catch (URISyntaxException e) {
+      throw new HiveException(e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerDesc.java
new file mode 100644
index 0000000..97870de
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerDesc.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for ALTER DATABASE ... SET OWNER ... commands.
+ */
+@Explain(displayName = "Set Database Owner", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class AlterDatabaseSetOwnerDesc extends AbstractAlterDatabaseDesc {
+  private static final long serialVersionUID = 1L;
+
+  private final PrincipalDesc ownerPrincipal;
+
+  public AlterDatabaseSetOwnerDesc(String databaseName, PrincipalDesc ownerPrincipal, ReplicationSpec replicationSpec) {
+    super(databaseName, replicationSpec);
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  @Explain(displayName="owner")
+  public PrincipalDesc getOwnerPrincipal() {
+    return ownerPrincipal;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerOperation.java
new file mode 100644
index 0000000..9b20311
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetOwnerOperation.java
@@ -0,0 +1,39 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+
+/**
+ * Operation process of altering a database's owner.
+ */
+public class AlterDatabaseSetOwnerOperation extends AbstractAlterDatabaseOperation<AlterDatabaseSetOwnerDesc> {
+  public AlterDatabaseSetOwnerOperation(DDLOperationContext context, AlterDatabaseSetOwnerDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  protected void doAlteration(Database database, Map<String, String> params) {
+    database.setOwnerName(desc.getOwnerPrincipal().getName());
+    database.setOwnerType(desc.getOwnerPrincipal().getType());
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesDesc.java
new file mode 100644
index 0000000..1a2075b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesDesc.java
@@ -0,0 +1,46 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for ALTER DATABASE ... SET PROPERTIES ... commands.
+ */
+@Explain(displayName = "Set Database Properties", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class AlterDatabaseSetPropertiesDesc extends AbstractAlterDatabaseDesc {
+  private static final long serialVersionUID = 1L;
+
+  private final Map<String, String> dbProperties;
+
+  public AlterDatabaseSetPropertiesDesc(String databaseName, Map<String, String> dbProperties,
+      ReplicationSpec replicationSpec) {
+    super(databaseName, replicationSpec);
+    this.dbProperties = dbProperties;
+  }
+
+  @Explain(displayName="properties")
+  public Map<String, String> getDatabaseProperties() {
+    return dbProperties;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java
new file mode 100644
index 0000000..7f73502
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hadoop.hive.ql.ddl.database;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+
+/**
+ * Operation process of altering a database's properties.
+ */
+public class AlterDatabaseSetPropertiesOperation
+  extends AbstractAlterDatabaseOperation<AlterDatabaseSetPropertiesDesc> {
+  public AlterDatabaseSetPropertiesOperation(DDLOperationContext context, AlterDatabaseSetPropertiesDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  protected void doAlteration(Database database, Map<String, String> params) {
+    Map<String, String> newParams = desc.getDatabaseProperties();
+
+    // if both old and new params are not null, merge them
+    if (params != null && newParams != null) {
+      params.putAll(newParams);
+      database.setParameters(params);
+    } else {
+      // if one of them is null, replace the old params with the new one
+      database.setParameters(newParams);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
index ac5f375..2a5966b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
@@ -21,7 +21,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
-import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetOwnerDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.database.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -131,8 +132,8 @@ public class LoadDatabase {
   }
 
   private Task<? extends Serializable> setOwnerInfoTask(Database dbObj) {
-    AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(dbObj.getName(), new PrincipalDesc(dbObj.getOwnerName(),
-        dbObj.getOwnerType()), null);
+    AlterDatabaseSetOwnerDesc alterDbDesc = new AlterDatabaseSetOwnerDesc(dbObj.getName(),
+        new PrincipalDesc(dbObj.getOwnerName(), dbObj.getOwnerType()), null);
     DDLWork work = new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc);
     return TaskFactory.get(work, context.hiveConf);
   }
@@ -161,7 +162,7 @@ public class LoadDatabase {
 
   private static Task<? extends Serializable> alterDbTask(String dbName, Map<String, String> props,
                                                           HiveConf hiveConf) {
-    AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(dbName, props, null);
+    AlterDatabaseSetPropertiesDesc alterDbDesc = new AlterDatabaseSetPropertiesDesc(dbName, props, null);
     DDLWork work = new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc);
     return TaskFactory.get(work, hiveConf);
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
index e3480d3..e83b9f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ReplLastIdInfo;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
-import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.misc.ReplRemoveFirstIncLoadPendFlagDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.AlterTableSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -278,7 +278,8 @@ public class IncrementalLoadTasksBuilder {
     HashMap<String, String> mapProp = new HashMap<>();
     mapProp.put(ReplicationSpec.KEY.CURR_STATE_ID.toString(), replState);
 
-    AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(dbName, mapProp, new ReplicationSpec(replState, replState));
+    AlterDatabaseSetPropertiesDesc alterDbDesc = new AlterDatabaseSetPropertiesDesc(dbName, mapProp,
+        new ReplicationSpec(replState, replState));
     Task<? extends Serializable> updateReplIdTask = TaskFactory.get(new DDLWork(inputs, outputs, alterDbDesc), conf);
 
     // Link the update repl state task with dependency collection task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index bd4edb7..698d7fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -74,7 +74,10 @@ import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc.DDLDescWithWriteId;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
-import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AbstractAlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetLocationDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetOwnerDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.database.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.database.DescDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.database.DropDatabaseDesc;
@@ -811,11 +814,11 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         throw new SemanticException("Unrecognized token in CREATE DATABASE statement");
       }
     }
-    AlterDatabaseDesc alterDesc = new AlterDatabaseDesc(dbName, dbProps, null);
+    AlterDatabaseSetPropertiesDesc alterDesc = new AlterDatabaseSetPropertiesDesc(dbName, dbProps, null);
     addAlterDbDesc(alterDesc);
   }
 
-  private void addAlterDbDesc(AlterDatabaseDesc alterDesc) throws SemanticException {
+  private void addAlterDbDesc(AbstractAlterDatabaseDesc alterDesc) throws SemanticException {
     Database database = getDatabase(alterDesc.getDatabaseName());
     outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_NO_LOCK));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterDesc)));
@@ -835,7 +838,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException("Owner type " + nullCmdMsg);
     }
 
-    AlterDatabaseDesc alterDesc = new AlterDatabaseDesc(dbName, principalDesc, null);
+    AlterDatabaseSetOwnerDesc alterDesc = new AlterDatabaseSetOwnerDesc(dbName, principalDesc, null);
     addAlterDbDesc(alterDesc);
   }
 
@@ -843,7 +846,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     String dbName = getUnescapedName((ASTNode) ast.getChild(0));
     String newLocation = unescapeSQLString(ast.getChild(1).getText());
     addLocationToOutputs(newLocation);
-    AlterDatabaseDesc alterDesc = new AlterDatabaseDesc(dbName, newLocation);
+    AlterDatabaseSetLocationDesc alterDesc = new AlterDatabaseSetLocationDesc(dbName, newLocation);
     addAlterDbDesc(alterDesc);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
index 6305754..189e9aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
@@ -21,7 +21,9 @@ import org.apache.hadoop.hive.metastore.ReplChangeManager;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.messaging.AlterDatabaseMessage;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
-import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AbstractAlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetOwnerDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -50,7 +52,7 @@ public class AlterDatabaseHandler extends AbstractMessageHandler {
     try {
       Database oldDb = msg.getDbObjBefore();
       Database newDb = msg.getDbObjAfter();
-      AlterDatabaseDesc alterDbDesc;
+      AbstractAlterDatabaseDesc alterDbDesc;
 
       if ((oldDb.getOwnerType() == newDb.getOwnerType())
             && oldDb.getOwnerName().equalsIgnoreCase(newDb.getOwnerName())) {
@@ -70,10 +72,10 @@ public class AlterDatabaseHandler extends AbstractMessageHandler {
           }
           newDbProps.put(key, entry.getValue());
         }
-        alterDbDesc = new AlterDatabaseDesc(actualDbName, newDbProps, context.eventOnlyReplicationSpec());
+        alterDbDesc = new AlterDatabaseSetPropertiesDesc(actualDbName, newDbProps, context.eventOnlyReplicationSpec());
       } else {
-        alterDbDesc = new AlterDatabaseDesc(actualDbName, new PrincipalDesc(newDb.getOwnerName(), newDb.getOwnerType()),
-                context.eventOnlyReplicationSpec());
+        alterDbDesc = new AlterDatabaseSetOwnerDesc(actualDbName, new PrincipalDesc(newDb.getOwnerName(),
+            newDb.getOwnerType()), context.eventOnlyReplicationSpec());
       }
 
       Task<DDLWork> alterDbTask = TaskFactory.get(
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
index eb4d8b4..1438a52 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
@@ -24,7 +24,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
-import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetOwnerDesc;
+import org.apache.hadoop.hive.ql.ddl.database.AlterDatabaseSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.database.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -61,14 +62,14 @@ public class CreateDatabaseHandler extends AbstractMessageHandler {
     Task<DDLWork> createDBTask = TaskFactory.get(
         new DDLWork(new HashSet<>(), new HashSet<>(), createDatabaseDesc), context.hiveConf);
     if (!db.getParameters().isEmpty()) {
-      AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(destinationDBName, db.getParameters(),
-          context.eventOnlyReplicationSpec());
+      AlterDatabaseSetPropertiesDesc alterDbDesc = new AlterDatabaseSetPropertiesDesc(destinationDBName,
+          db.getParameters(), context.eventOnlyReplicationSpec());
       Task<DDLWork> alterDbProperties = TaskFactory
           .get(new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc), context.hiveConf);
       createDBTask.addDependentTask(alterDbProperties);
     }
     if (StringUtils.isNotEmpty(db.getOwnerName())) {
-      AlterDatabaseDesc alterDbOwner = new AlterDatabaseDesc(destinationDBName,
+      AlterDatabaseSetOwnerDesc alterDbOwner = new AlterDatabaseSetOwnerDesc(destinationDBName,
           new PrincipalDesc(db.getOwnerName(), db.getOwnerType()),
           context.eventOnlyReplicationSpec());
       Task<DDLWork> alterDbTask = TaskFactory
diff --git a/ql/src/test/queries/clientpositive/alter_db_owner.q b/ql/src/test/queries/clientpositive/alter_db_owner.q
index b224f33..94c7511 100644
--- a/ql/src/test/queries/clientpositive/alter_db_owner.q
+++ b/ql/src/test/queries/clientpositive/alter_db_owner.q
@@ -2,6 +2,7 @@
 create database db_alter_onr;
 describe database db_alter_onr;
 
+explain alter database db_alter_onr set owner user user1;
 alter database db_alter_onr set owner user user1;
 describe database db_alter_onr;
 
diff --git a/ql/src/test/results/clientpositive/alter_db_owner.q.out b/ql/src/test/results/clientpositive/alter_db_owner.q.out
index bc3752f..de4ba64 100644
--- a/ql/src/test/results/clientpositive/alter_db_owner.q.out
+++ b/ql/src/test/results/clientpositive/alter_db_owner.q.out
@@ -17,6 +17,23 @@ PREHOOK: Output: database:db_alter_onr
 #### A masked pattern was here ####
 POSTHOOK: type: ALTERDATABASE_OWNER
 POSTHOOK: Output: database:db_alter_onr
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+#### A masked pattern was here ####
+      name: db_alter_onr
+#### A masked pattern was here ####
+        Principal
+          name: user1
+
+#### A masked pattern was here ####
+PREHOOK: type: ALTERDATABASE_OWNER
+PREHOOK: Output: database:db_alter_onr
+#### A masked pattern was here ####
+POSTHOOK: type: ALTERDATABASE_OWNER
+POSTHOOK: Output: database:db_alter_onr
 PREHOOK: query: describe database db_alter_onr
 PREHOOK: type: DESCDATABASE
 PREHOOK: Input: database:db_alter_onr
diff --git a/ql/src/test/results/clientpositive/db_ddl_explain.q.out b/ql/src/test/results/clientpositive/db_ddl_explain.q.out
index 8e85d75..8cf5f9c 100644
--- a/ql/src/test/results/clientpositive/db_ddl_explain.q.out
+++ b/ql/src/test/results/clientpositive/db_ddl_explain.q.out
@@ -81,7 +81,7 @@ STAGE DEPENDENCIES:
 
 STAGE PLANS:
   Stage: Stage-0
-    Alter Database
+    Set Database Properties
       name: d
       properties:
         test yesthisis