[hive] branch master updated: [Addendum: missed files] HIVE-23118: Option for exposing compile time counters as tez counters (Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)

2020-04-03 Thread prasanthj
This is an automated email from the ASF dual-hosted git repository.

prasanthj 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 e89995a  [Addendum: missed files] HIVE-23118: Option for exposing 
compile time counters as tez counters (Prasanth Jayachandran reviewed by Jesus 
Camacho Rodriguez)
e89995a is described below

commit e89995afe1de914caa86ef99c54993ef809d9402
Author: Prasanth Jayachandran 
AuthorDate: Fri Apr 3 18:11:24 2020 -0700

[Addendum: missed files] HIVE-23118: Option for exposing compile time 
counters as tez counters (Prasanth Jayachandran reviewed by Jesus Camacho 
Rodriguez)

Signed-off-by: Prasanth Jayachandran 
---
 .../hive/ql/exec/tez/CompileTimeCounters.java  |   27 +
 .../hive/ql/hooks/CompileTimeCounterPreHook.java   |   84 ++
 .../queries/clientpositive/tez_compile_counters.q  |   27 +
 .../clientpositive/llap/tez_compile_counters.q.out | 1540 
 4 files changed, 1678 insertions(+)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CompileTimeCounters.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CompileTimeCounters.java
new file mode 100644
index 000..162643c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CompileTimeCounters.java
@@ -0,0 +1,27 @@
+/*
+ * 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.exec.tez;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public enum CompileTimeCounters {
+  TOTAL_FILE_SIZE,
+  RAW_DATA_SIZE,
+}
\ No newline at end of file
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/hooks/CompileTimeCounterPreHook.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/CompileTimeCounterPreHook.java
new file mode 100644
index 000..415ac42
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/CompileTimeCounterPreHook.java
@@ -0,0 +1,84 @@
+/*
+ * 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.hooks;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.CompileTimeCounters;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.hooks.HookContext.HookType;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.tez.common.counters.TezCounters;
+
+/**
+ * Implementation of a pre execute hook that adds compile time tez counters to 
tez tasks.
+ */
+public class CompileTimeCounterPreHook implements ExecuteWithHookContext {
+
+  @Override
+  public void run(HookContext hookContext) throws Exception {
+assert(hookContext.getHookType() == HookType.PRE_EXEC_HOOK);
+QueryPlan plan = hookContext.getQueryPlan();
+if (plan == null) {
+  return;
+}
+
+int numMrJobs = Utilities.getMRTasks(plan.getRootTasks()).size();
+List rootTasks = Utilities.getTezTasks(plan.getRootTasks());
+int numTezJobs = rootTasks.size();
+if (numMrJobs + numTezJobs <= 0) {
+  return; // ignore client only queries
+}
+
+for (TezTask 

[hive] branch master updated: HIVE-23118: Option for exposing compile time counters as tez counters (Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)

2020-04-03 Thread prasanthj
This is an automated email from the ASF dual-hosted git repository.

prasanthj 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 265f183  HIVE-23118: Option for exposing compile time counters as tez 
counters (Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)
265f183 is described below

commit 265f183338c786e47e72d9617dfef3c3023b7a4f
Author: Prasanth Jayachandran 
AuthorDate: Fri Apr 3 18:08:22 2020 -0700

HIVE-23118: Option for exposing compile time counters as tez counters 
(Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)

Signed-off-by: Prasanth Jayachandran 
---
 .../test/resources/testconfiguration.properties|  1 +
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java| 10 +++--
 .../org/apache/hadoop/hive/ql/exec/tez/Utils.java  | 26 ++
 .../hive/ql/exec/tez/monitoring/TezJobMonitor.java | 12 +++---
 .../hive/ql/hooks/PostExecTezSummaryPrinter.java   |  6 +
 5 files changed, 50 insertions(+), 5 deletions(-)

diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index 3e89071..f54c96e 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -689,6 +689,7 @@ minillaplocal.query.files=\
   orc_llap.q,\
   orc_llap_nonvector.q,\
   orc_ppd_date.q,\
+  tez_compile_counters.q,\
   tez_input_counters.q,\
   orc_ppd_decimal.q,\
   orc_ppd_timestamp.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 0bacb05..3599d19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -127,6 +127,9 @@ public class TezTask extends Task {
 return counters;
   }
 
+  public void setTezCounters(final TezCounters counters) {
+this.counters = counters;
+  }
 
   @Override
   public int execute() {
@@ -235,7 +238,7 @@ public class TezTask extends Task {
 }
 
 // finally monitor will print progress until the job is done
-TezJobMonitor monitor = new TezJobMonitor(work.getAllWork(), 
dagClient, conf, dag, ctx);
+TezJobMonitor monitor = new TezJobMonitor(work.getAllWork(), 
dagClient, conf, dag, ctx, counters);
 rc = monitor.monitorExecution();
 
 if (rc != 0) {
@@ -245,7 +248,10 @@ public class TezTask extends Task {
 // fetch the counters
 try {
   Set statusGetOpts = 
EnumSet.of(StatusGetOpts.GET_COUNTERS);
-  counters = dagClient.getDAGStatus(statusGetOpts).getDAGCounters();
+  TezCounters dagCounters = 
dagClient.getDAGStatus(statusGetOpts).getDAGCounters();
+  // if initial counters exists, merge it with dag counters to get 
aggregated view
+  TezCounters mergedCounters = counters == null ? dagCounters : 
Utils.mergeTezCounters(dagCounters, counters);
+  counters = mergedCounters;
 } catch (Exception err) {
   // Don't fail execution due to counters - just don't print summary 
info
   LOG.warn("Failed to get counters. Ignoring, summary info will be 
incomplete. " + err, err);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
index cc9c4ce..d266bb1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
@@ -30,6 +30,7 @@ import 
org.apache.hadoop.hive.llap.registry.LlapServiceInstance;
 import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.split.SplitLocationProvider;
+import org.apache.tez.common.counters.TezCounters;
 import org.slf4j.Logger;
 
 public class Utils {
@@ -101,4 +102,29 @@ public class Utils {
 }
 return new HostAffinitySplitLocationProvider(locations);
   }
+
+
+  /**
+   * Merges two different tez counters into one
+   *
+   * @param counter1 - tez counter 1
+   * @param counter2 - tez counter 2
+   * @return - merged tez counter
+   */
+  public static TezCounters mergeTezCounters(final TezCounters counter1, final 
TezCounters counter2) {
+TezCounters merged = new TezCounters();
+if (counter1 != null) {
+  for (String counterGroup : counter1.getGroupNames()) {
+merged.addGroup(counter1.getGroup(counterGroup));
+  }
+}
+
+if (counter2 != null) {
+  for (String counterGroup : counter2.getGroupNames()) {
+merged.addGroup(counter2.getGroup(counterGroup));
+  }
+}
+
+return merged;
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java 

[hive] branch master updated: HIVE-23130: User friendly error message when MV rewriting fails (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)

2020-04-03 Thread jcamacho
This is an automated email from the ASF dual-hosted git repository.

jcamacho 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 505fd69  HIVE-23130: User friendly error message when MV rewriting 
fails (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
505fd69 is described below

commit 505fd6935540e234fb28bcc7e42991f81b1951ac
Author: Krisztian Kasa 
AuthorDate: Fri Apr 3 16:27:57 2020 -0700

HIVE-23130: User friendly error message when MV rewriting fails (Krisztian 
Kasa, reviewed by Jesus Camacho Rodriguez)
---
 .../calcite/HiveRelOptMaterializationValidator.java   | 15 ++-
 .../clientnegative/materialized_view_no_cbo_rewrite.q.out |  2 +-
 .../materialized_view_no_cbo_rewrite_2.q.out  |  2 +-
 3 files changed, 16 insertions(+), 3 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
index 110136d..1aa1731 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
@@ -46,6 +46,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
+import 
org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
@@ -123,7 +124,6 @@ public class HiveRelOptMaterializationValidator extends 
HiveRelShuttleImpl {
 
   @Override
   public RelNode visit(RelNode node) {
-setAutomaticRewritingInvalidReason(node);
 // There are several Hive RelNode types which do not have their own 
visit() method
 // defined in the HiveRelShuttle interface, which need to be handled 
appropriately here.
 // Per jcamachorodriguez we should not encounter 
HiveMultiJoin/HiveSortExchange
@@ -132,6 +132,8 @@ public class HiveRelOptMaterializationValidator extends 
HiveRelShuttleImpl {
   return visit((HiveUnion) node);
 } else if (node instanceof HiveSortLimit) {
   return visit((HiveSortLimit) node);
+} else if (node instanceof HiveSortExchange) {
+  return visit((HiveSortExchange) node);
 } else if (node instanceof HiveSemiJoin) {
   return visit((HiveSemiJoin) node);
 } else if (node instanceof HiveExcept) {
@@ -225,18 +227,27 @@ public class HiveRelOptMaterializationValidator extends 
HiveRelShuttleImpl {
 
   // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveUnion union) {
+setAutomaticRewritingInvalidReason("Statement has unsupported operator: 
union.");
 return visitChildren(union);
   }
 
   // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveSortLimit sort) {
+setAutomaticRewritingInvalidReason("Statement has unsupported clause: 
order by.");
 checkExpr(sort.getFetchExpr());
 checkExpr(sort.getOffsetExpr());
 return visitChildren(sort);
   }
 
   // Note: Not currently part of the HiveRelNode interface
+  private RelNode visit(HiveSortExchange sort) {
+setAutomaticRewritingInvalidReason("Statement has unsupported clause: sort 
by.");
+return visitChildren(sort);
+  }
+
+  // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveSemiJoin semiJoin) {
+setAutomaticRewritingInvalidReason("Statement has unsupported join type: 
semi join.");
 checkExpr(semiJoin.getCondition());
 checkExpr(semiJoin.getJoinFilter());
 return visitChildren(semiJoin);
@@ -244,11 +255,13 @@ public class HiveRelOptMaterializationValidator extends 
HiveRelShuttleImpl {
 
   // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveExcept except) {
+setAutomaticRewritingInvalidReason("Statement has unsupported operator: 
except.");
 return visitChildren(except);
   }
 
   // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveIntersect intersect) {
+setAutomaticRewritingInvalidReason("Statement has unsupported operator: 
intersect.");
 return visitChildren(intersect);
   }
 
diff --git 
a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out 
b/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out
index 8e55c2c..159e2dc 100644
--- 

[hive] branch master updated: HIVE-23132: Add test of Explain CBO of Merge statements (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)

2020-04-03 Thread jcamacho
This is an automated email from the ASF dual-hosted git repository.

jcamacho 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 dea35b4  HIVE-23132: Add test of Explain CBO of Merge statements 
(Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
dea35b4 is described below

commit dea35b4fd65fc6b4573133aa0b83000bcddd42b6
Author: Krisztian Kasa 
AuthorDate: Fri Apr 3 10:00:53 2020 -0700

HIVE-23132: Add test of Explain CBO of Merge statements (Krisztian Kasa, 
reviewed by Jesus Camacho Rodriguez)
---
 ql/src/test/queries/clientpositive/sort_acid.q | 13 
 .../results/clientpositive/llap/sort_acid.q.out| 71 ++
 2 files changed, 84 insertions(+)

diff --git a/ql/src/test/queries/clientpositive/sort_acid.q 
b/ql/src/test/queries/clientpositive/sort_acid.q
index bfa14bd..2793250 100644
--- a/ql/src/test/queries/clientpositive/sort_acid.q
+++ b/ql/src/test/queries/clientpositive/sort_acid.q
@@ -48,3 +48,16 @@ where a in (
  where o.d = 21);
 
 select * from acidtlb;
+
+explain cbo
+merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 60 then delete
+when matched and s.c = 60 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d);
+
+merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 30 then delete
+when matched and s.c = 30 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d);
+
+select * from acidtlb;
diff --git a/ql/src/test/results/clientpositive/llap/sort_acid.q.out 
b/ql/src/test/results/clientpositive/llap/sort_acid.q.out
index 46618e6..d285abd 100644
--- a/ql/src/test/results/clientpositive/llap/sort_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/sort_acid.q.out
@@ -249,3 +249,74 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@acidtlb
  A masked pattern was here 
 30 450
+PREHOOK: query: explain cbo
+merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 60 then delete
+when matched and s.c = 60 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acidtlb
+PREHOOK: Input: default@othertlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@merge_tmp_table
+POSTHOOK: query: explain cbo
+merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 60 then delete
+when matched and s.c = 60 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@acidtlb
+POSTHOOK: Input: default@othertlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@merge_tmp_table
+CBO PLAN:
+HiveProject(col0=[$3], col1=[$0], col2=[$1], col3=[$2])
+  HiveJoin(condition=[=($2, $0)], joinType=[left], algorithm=[none], cost=[not 
available])
+HiveProject(c=[$0], d=[$1])
+  HiveTableScan(table=[[default, othertlb]], table:alias=[s])
+HiveProject(a=[$0], ROW__ID=[$4])
+  HiveFilter(condition=[IS NOT NULL($0)])
+HiveTableScan(table=[[default, acidtlb]], table:alias=[t])
+
+PREHOOK: query: merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 30 then delete
+when matched and s.c = 30 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acidtlb
+PREHOOK: Input: default@othertlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@merge_tmp_table
+POSTHOOK: query: merge into acidtlb as t using othertlb as s on t.a = s.c
+when matched and s.c < 30 then delete
+when matched and s.c = 30 then update set b = 1000
+when not matched then insert values (s.c, 2000 + s.d)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@acidtlb
+POSTHOOK: Input: default@othertlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Lineage: acidtlb.a SIMPLE [(othertlb)s.FieldSchema(name:c, type:int, 
comment:null), ]
+POSTHOOK: Lineage: acidtlb.b EXPRESSION [(othertlb)s.FieldSchema(name:d, 
type:int, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION 
[(acidtlb)t.FieldSchema(name:ROW__ID, 
type:struct, comment:), ]
+PREHOOK: query: select * from acidtlb
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acidtlb
+ A masked pattern was here 
+POSTHOOK: query: select * from acidtlb
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@acidtlb
+ A masked pattern was here 
+10 2021
+60 2023
+70 2024
+80 2025
+30 1000



[hive] branch master updated: HIVE-22995: Add support for location for managed tables on database (Naveen Gangam, reviewed by Thejas Nair, Sam An)

2020-04-03 Thread ngangam
This is an automated email from the ASF dual-hosted git repository.

ngangam 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 3ab174d  HIVE-22995: Add support for location for managed tables on 
database (Naveen Gangam, reviewed by Thejas Nair, Sam An)
3ab174d is described below

commit 3ab174d82ffc2bd27432c0b04433be3bd7db5c6a
Author: Naveen Gangam 
AuthorDate: Fri Apr 3 02:48:07 2020 -0400

HIVE-22995: Add support for location for managed tables on database (Naveen 
Gangam, reviewed by Thejas Nair, Sam An)
---
 .../hive/hcatalog/mapreduce/HCatMapReduceTest.java |   6 +-
 .../metastore/TestHiveMetastoreTransformer.java|   4 +-
 .../metastore/TestTenantBasedStorageHierarchy.java | 598 +
 .../hadoop/hive/ql/TestWarehouseExternalDir.java   |   2 +-
 .../parse/BaseReplicationScenariosAcidTables.java  |   1 +
 .../parse/TestReplicationScenariosAcidTables.java  |   2 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g|   1 +
 .../org/apache/hadoop/hive/ql/parse/HiveParser.g   |  22 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g   |   2 +-
 .../location/AlterDatabaseSetLocationDesc.java |  20 +-
 .../AlterDatabaseSetLocationOperation.java |  31 +-
 .../AlterDatabaseSetManagedLocationAnalyzer.java   |  47 ++
 .../database/create/CreateDatabaseAnalyzer.java|  10 +-
 .../ql/ddl/database/create/CreateDatabaseDesc.java |  12 +
 .../database/create/CreateDatabaseOperation.java   |  21 +-
 .../ql/ddl/database/desc/DescDatabaseDesc.java |   2 +-
 .../ddl/database/desc/DescDatabaseOperation.java   |   2 +-
 .../org/apache/hadoop/hive/ql/exec/Utilities.java  |   5 +
 .../metadata/formatting/JsonMetaDataFormatter.java |   5 +-
 .../ql/metadata/formatting/MetaDataFormatter.java  |   2 +-
 .../metadata/formatting/TextMetaDataFormatter.java |   6 +-
 .../apache/hadoop/hive/ql/plan/HiveOperation.java  |   3 +-
 .../clientpositive/alter_change_db_location.q.out  |   2 +-
 .../results/clientpositive/alter_db_owner.q.out|   6 +-
 .../authorization_owner_actions_db.q.out   |   2 +-
 .../clientpositive/beeline/escape_comments.q.out   |   4 +-
 .../results/clientpositive/database_location.q.out |   4 +-
 .../clientpositive/database_properties.q.out   |   6 +-
 .../results/clientpositive/db_ddl_explain.q.out|   2 +-
 .../results/clientpositive/describe_database.q.out |   4 +-
 .../results/clientpositive/escape_comments.q.out   |   4 +-
 .../clientpositive/tez/explainanalyze_3.q.out  |   2 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   2 +-
 .../results/clientpositive/unicode_comments.q.out  |   2 +-
 .../apache/hadoop/hive/metastore/api/Database.java | 114 +++-
 .../src/gen/thrift/gen-php/metastore/Types.php |  23 +
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |  15 +-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |   4 +-
 .../apache/hadoop/hive/metastore/Warehouse.java|  77 ++-
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |   5 +
 .../src/main/thrift/hive_metastore.thrift  |   1 +
 .../hadoop/hive/metastore/HiveAlterHandler.java|   4 +-
 .../hadoop/hive/metastore/MetaStoreDirectSql.java  |   3 +-
 .../metastore/MetastoreDefaultTransformer.java | 148 +++--
 .../apache/hadoop/hive/metastore/ObjectStore.java  |   5 +
 .../metastore/client/builder/DatabaseBuilder.java  |  11 +-
 .../hadoop/hive/metastore/model/MDatabase.java |  32 ++
 .../src/main/resources/package.jdo |   4 +
 .../src/main/sql/derby/hive-schema-4.0.0.derby.sql |   3 +-
 .../sql/derby/upgrade-3.2.0-to-4.0.0.derby.sql |   5 +-
 .../src/main/sql/mssql/hive-schema-4.0.0.mssql.sql |   3 +-
 .../sql/mssql/upgrade-3.2.0-to-4.0.0.mssql.sql |   2 +
 .../src/main/sql/mysql/hive-schema-4.0.0.mysql.sql |   1 +
 .../sql/mysql/upgrade-3.2.0-to-4.0.0.mysql.sql |   3 +
 .../main/sql/oracle/hive-schema-4.0.0.oracle.sql   |   3 +-
 .../sql/oracle/upgrade-3.2.0-to-4.0.0.oracle.sql   |   3 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql|   3 +-
 .../postgres/upgrade-3.2.0-to-4.0.0.postgres.sql   |   3 +
 .../hive/metastore/TestCatalogOldClient.java   |   2 +-
 .../hadoop/hive/metastore/TestHiveMetaStore.java   |  14 +-
 .../hive/metastore/client/TestDatabases.java   |   2 +-
 .../client/TestTablesCreateDropAlterTruncate.java  |   4 +-
 .../minihms/AbstractMetaStoreService.java  |   1 +
 .../schematool/TestSchemaToolForMetastore.java |  18 +-
 64 files changed, 1223 insertions(+), 137 deletions(-)

diff --git 
a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java
 
b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java
index 983a66a..a3bc2d5 100644
--- 
a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java
+++