You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2018/03/31 01:25:30 UTC

[11/44] hive git commit: HIVE-18755 Modifications to the metastore for catalogs (Alan Gates, reviewed by Thejas Nair)

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
index b963f78..aa014e9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
@@ -55,12 +55,13 @@ public class InsertEvent extends ListenerEvent {
    * @param status status of insert, true = success, false = failure
    * @param handler handler that is firing the event
    */
-  public InsertEvent(String db, String table, List<String> partVals,
+  public InsertEvent(String catName, String db, String table, List<String> partVals,
       InsertEventRequestData insertData, boolean status, IHMSHandler handler) throws MetaException,
       NoSuchObjectException {
     super(status, handler);
 
     GetTableRequest req = new GetTableRequest(db, table);
+    req.setCatName(catName);
     // TODO MS-SPLIT Switch this back once HiveMetaStoreClient is moved.
     //req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
     req.setCapabilities(new ClientCapabilities(
@@ -68,7 +69,8 @@ public class InsertEvent extends ListenerEvent {
     try {
       this.tableObj = handler.get_table_req(req).getTable();
       if (partVals != null) {
-        this.ptnObj = handler.get_partition(db, table, partVals);
+        this.ptnObj = handler.get_partition(MetaStoreUtils.prependNotNullCatToDbName(catName, db),
+            table, partVals);
       } else {
         this.ptnObj = null;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java
new file mode 100644
index 0000000..96aa22c
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreCreateCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.CREATE_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java
new file mode 100644
index 0000000..0e01ccd
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreDropCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.DROP_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
index 7ddb8fe..b45a537 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
@@ -50,7 +50,10 @@ public abstract class PreEventContext {
     ALTER_SCHEMA_VERSION,
     DROP_SCHEMA_VERSION,
     READ_ISCHEMA,
-    READ_SCHEMA_VERSION
+    READ_SCHEMA_VERSION,
+    CREATE_CATALOG,
+    DROP_CATALOG,
+    READ_CATALOG
   }
 
   private final PreEventType eventType;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
index 999ec31..a380301 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
@@ -28,18 +28,24 @@ import java.util.Map;
 @InterfaceStability.Stable
 public class PreLoadPartitionDoneEvent extends PreEventContext {
 
+  private final String catName;
   private final String dbName;
   private final String tableName;
   private final Map<String,String> partSpec;
 
-  public PreLoadPartitionDoneEvent(String dbName, String tableName,
+  public PreLoadPartitionDoneEvent(String catName, String dbName, String tableName,
       Map<String, String> partSpec, IHMSHandler handler) {
     super(PreEventType.LOAD_PARTITION_DONE, handler);
+    this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.partSpec = partSpec;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java
new file mode 100644
index 0000000..3f1afdf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreReadCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreReadCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.READ_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
new file mode 100644
index 0000000..cbb0f4e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
@@ -0,0 +1,25 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.messaging;
+
+public abstract class CreateCatalogMessage extends EventMessage {
+
+  protected CreateCatalogMessage() {
+    super(EventType.CREATE_CATALOG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
new file mode 100644
index 0000000..0e731ce
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
@@ -0,0 +1,25 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.messaging;
+
+public abstract class DropCatalogMessage extends EventMessage {
+
+  protected DropCatalogMessage() {
+    super(EventType.DROP_CATALOG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
index 8578d4a..3cbfa55 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -53,7 +53,9 @@ public abstract class EventMessage {
     DROP_ISCHEMA(MessageFactory.DROP_ISCHEMA_EVENT),
     ADD_SCHEMA_VERSION(MessageFactory.ADD_SCHEMA_VERSION_EVENT),
     ALTER_SCHEMA_VERSION(MessageFactory.ALTER_SCHEMA_VERSION_EVENT),
-    DROP_SCHEMA_VERSION(MessageFactory.DROP_SCHEMA_VERSION_EVENT);
+    DROP_SCHEMA_VERSION(MessageFactory.DROP_SCHEMA_VERSION_EVENT),
+    CREATE_CATALOG(MessageFactory.CREATE_CATALOG_EVENT),
+    DROP_CATALOG(MessageFactory.DROP_CATALOG_EVENT);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index 5976c48..ab93f82 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hive.metastore.messaging;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -65,6 +66,8 @@ public abstract class MessageFactory {
   public static final String ADD_SCHEMA_VERSION_EVENT = "ADD_SCHEMA_VERSION";
   public static final String ALTER_SCHEMA_VERSION_EVENT = "ALTER_SCHEMA_VERSION";
   public static final String DROP_SCHEMA_VERSION_EVENT = "DROP_SCHEMA_VERSION";
+  public static final String CREATE_CATALOG_EVENT = "CREATE_CATALOG";
+  public static final String DROP_CATALOG_EVENT = "DROP_CATALOG";
 
 
   private static MessageFactory instance = null;
@@ -276,4 +279,8 @@ public abstract class MessageFactory {
    */
   public abstract DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
       String constraintName);
+
+  public abstract CreateCatalogMessage buildCreateCatalogMessage(Catalog catalog);
+
+  public abstract DropCatalogMessage buildDropCatalogMessage(Catalog catalog);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
new file mode 100644
index 0000000..8a26764
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
@@ -0,0 +1,79 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.CreateCatalogMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONCreateCatalogMessage extends CreateCatalogMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, catalog;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Required for Jackson
+   */
+  public JSONCreateCatalogMessage() {
+
+  }
+
+  public JSONCreateCatalogMessage(String server, String servicePrincipal, String catalog,
+                                  Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.catalog = catalog;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  public String getCatalog() {
+    return catalog;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
new file mode 100644
index 0000000..58e95f4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
@@ -0,0 +1,66 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.DropCatalogMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONDropCatalogMessage extends DropCatalogMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, catalog;
+
+  @JsonProperty
+  Long timestamp;
+
+  public JSONDropCatalogMessage() {
+
+  }
+
+  public JSONDropCatalogMessage(String server, String servicePrincipal, String catalog,
+                                Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.catalog = catalog;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  public String getCatalog() {
+    return catalog;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index 4f03a27..0fc5387 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -28,6 +28,7 @@ import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
 
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
@@ -45,9 +46,11 @@ import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateCatalogMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropCatalogMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
@@ -190,6 +193,16 @@ public class JSONMessageFactory extends MessageFactory {
         constraintName, now());
   }
 
+  @Override
+  public CreateCatalogMessage buildCreateCatalogMessage(Catalog catalog) {
+    return new JSONCreateCatalogMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, catalog.getName(), now());
+  }
+
+  @Override
+  public DropCatalogMessage buildDropCatalogMessage(Catalog catalog) {
+    return new JSONDropCatalogMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, catalog.getName(), now());
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
new file mode 100644
index 0000000..e82cb43
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
@@ -0,0 +1,58 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.metastore.model;
+
+public class MCatalog {
+  private String name;
+  private String description;
+  private String locationUri;
+
+  public MCatalog() {
+
+  }
+
+  public MCatalog(String name, String description, String locationUri) {
+    this.name = name;
+    this.description = description;
+    this.locationUri = locationUri;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public String getLocationUri() {
+    return locationUri;
+  }
+
+  public void setLocationUri(String locationUri) {
+    this.locationUri = locationUri;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
index 1133cb1..66b5d48 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
@@ -27,6 +27,7 @@ import java.util.Set;
  */
 public class MCreationMetadata {
 
+  private String catalogName;
   private String dbName;
   private String tblName;
   private Set<MTable> tables;
@@ -35,8 +36,9 @@ public class MCreationMetadata {
   public MCreationMetadata() {
   }
 
-  public MCreationMetadata(String dbName, String tblName,
+  public MCreationMetadata(String catName, String dbName, String tblName,
       Set<MTable> tables, String txnList) {
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.tables = tables;
@@ -59,6 +61,14 @@ public class MCreationMetadata {
     this.txnList = txnList;
   }
 
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
index e8034ce..fa30330 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
@@ -34,6 +34,7 @@ public class MDatabase {
   private Map<String, String> parameters;
   private String ownerName;
   private String ownerType;
+  private String catalogName;
 
   /**
    * Default construction to keep jpox/jdo happy
@@ -46,12 +47,13 @@ public class MDatabase {
    * @param locationUri Location of the database in the warehouse
    * @param description Comment describing the database
    */
-  public MDatabase(String name, String locationUri, String description,
+  public MDatabase(String catalogName, String name, String locationUri, String description,
       Map<String, String> parameters) {
     this.name = name;
     this.locationUri = locationUri;
     this.description = description;
     this.parameters = parameters;
+    this.catalogName = catalogName;
   }
 
   /**
@@ -125,4 +127,12 @@ public class MDatabase {
   public void setOwnerType(String ownerType) {
     this.ownerType = ownerType;
   }
+
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
index 1b1f7fd..60914ae 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
@@ -22,6 +22,7 @@ public class MNotificationLog {
   private long eventId; // This is not the datanucleus id, but the id assigned by the sequence
   private int eventTime;
   private String eventType;
+  private String catalogName;
   private String dbName;
   private String tableName;
   private String message;
@@ -30,10 +31,11 @@ public class MNotificationLog {
   public MNotificationLog() {
   }
 
-  public MNotificationLog(int eventId, String eventType, String dbName, String tableName,
+  public MNotificationLog(int eventId, String eventType, String catName, String dbName, String tableName,
                           String message) {
     this.eventId = eventId;
     this.eventType = eventType;
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.message = message;
@@ -72,6 +74,14 @@ public class MNotificationLog {
     this.dbName = dbName;
   }
 
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   public String getTableName() {
     return tableName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
index f7ef6fc..50d9c5b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
@@ -34,6 +34,7 @@ public class MPartitionColumnStatistics {
 
   private MPartition partition;
 
+  private String catName;
   private String dbName;
   private String tableName;
   private String partitionName;
@@ -137,6 +138,14 @@ public class MPartitionColumnStatistics {
     this.dbName = dbName;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
   public MPartition getPartition() {
     return partition;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
index 50c5045..d0cc51a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hive.metastore.model;
 
 public class MPartitionEvent {
 
+  private String catalogName;
+
   private String dbName;
 
   private String tblName;
@@ -31,8 +33,9 @@ public class MPartitionEvent {
 
   private int eventType;
 
-  public MPartitionEvent(String dbName, String tblName, String partitionName, int eventType) {
+  public MPartitionEvent(String catName, String dbName, String tblName, String partitionName, int eventType) {
     super();
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.partName = partitionName;
@@ -42,6 +45,10 @@ public class MPartitionEvent {
 
   public MPartitionEvent() {}
 
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   /**
    * @param dbName the dbName to set
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
index ec61317..731cd6f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
@@ -33,6 +33,7 @@ package org.apache.hadoop.hive.metastore.model;
 public class MTableColumnStatistics {
 
   private MTable table;
+  private String catName;
   private String dbName;
   private String tableName;
   private String colName;
@@ -151,6 +152,14 @@ public class MTableColumnStatistics {
     this.dbName = dbName;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
   public void setBooleanStats(Long numTrues, Long numFalses, Long numNulls) {
     this.numTrues = numTrues;
     this.numFalses = numFalses;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
index e34335d..92813b9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
@@ -26,11 +26,14 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 /**
  * Implementation of PartitionSpecProxy that composes a list of PartitionSpecProxy.
  */
 public class CompositePartitionSpecProxy extends PartitionSpecProxy {
 
+  private String catName;
   private String dbName;
   private String tableName;
   private List<PartitionSpec> partitionSpecs;
@@ -40,10 +43,12 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   protected CompositePartitionSpecProxy(List<PartitionSpec> partitionSpecs) {
     this.partitionSpecs = partitionSpecs;
     if (partitionSpecs.isEmpty()) {
+      catName = null;
       dbName = null;
       tableName = null;
     }
     else {
+      catName = partitionSpecs.get(0).getCatName();
       dbName = partitionSpecs.get(0).getDbName();
       tableName = partitionSpecs.get(0).getTableName();
       this.partitionSpecProxies = new ArrayList<>(partitionSpecs.size());
@@ -57,7 +62,15 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
     assert isValid() : "Invalid CompositePartitionSpecProxy!";
   }
 
+  @Deprecated
   protected CompositePartitionSpecProxy(String dbName, String tableName, List<PartitionSpec> partitionSpecs) {
+    this(DEFAULT_CATALOG_NAME, dbName, tableName, partitionSpecs);
+
+  }
+
+  protected CompositePartitionSpecProxy(String catName, String dbName, String tableName,
+                                        List<PartitionSpec> partitionSpecs) {
+    this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.partitionSpecs = partitionSpecs;
@@ -146,6 +159,11 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
     }
 
     @Override
+    public String getCatName() {
+      return composite.getCatName();
+    }
+
+    @Override
     public String getDbName() {
       return composite.dbName;
     }
@@ -182,6 +200,15 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    this.catName = catName;
+    for (PartitionSpecProxy partSpecProxy : partitionSpecProxies) {
+      partSpecProxy.setCatName(catName);
+    }
+
+  }
+
+  @Override
   public void setDbName(String dbName) {
     this.dbName = dbName;
     for (PartitionSpecProxy partSpecProxy : partitionSpecProxies) {
@@ -198,6 +225,11 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return catName;
+  }
+
+  @Override
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
index 7b0550b..6bd29d0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
@@ -40,6 +40,11 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return partitionSpec.getCatName();
+  }
+
+  @Override
   public String getDbName() {
     return partitionSpec.getDbName();
   }
@@ -65,6 +70,14 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    partitionSpec.setCatName(catName);
+    for (Partition partition : partitionSpec.getPartitionList().getPartitions()) {
+      partition.setCatName(catName);
+    }
+  }
+
+  @Override
   public void setDbName(String dbName) {
     partitionSpec.setDbName(dbName);
     for (Partition partition : partitionSpec.getPartitionList().getPartitions()) {
@@ -118,6 +131,11 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
     }
 
     @Override
+    public String getCatName() {
+      return partitionSpecProxy.getCatName();
+    }
+
+    @Override
     public String getDbName() {
       return partitionSpecProxy.getDbName();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
index 2640a24..ff2dea1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
@@ -37,6 +37,12 @@ public abstract class PartitionSpecProxy {
   public abstract int size();
 
   /**
+   * Set catalog name.
+   * @param catName catalog name.
+   */
+  public abstract void setCatName(String catName);
+
+  /**
    * Setter for name of the DB.
    * @param dbName The name of the DB.
    */
@@ -49,6 +55,12 @@ public abstract class PartitionSpecProxy {
   public abstract void setTableName(String tableName);
 
   /**
+   * Get catalog name.
+   * @return catalog name.
+   */
+  public abstract String getCatName();
+
+  /**
    * Getter for name of the DB.
    * @return The name of the DB.
    */
@@ -131,6 +143,12 @@ public abstract class PartitionSpecProxy {
     Partition getCurrent();
 
     /**
+     * Get the catalog name.
+     * @return catalog name.
+     */
+    String getCatName();
+
+    /**
      * Getter for the name of the DB.
      * @return Name of the DB.
      */
@@ -184,6 +202,7 @@ public abstract class PartitionSpecProxy {
     public SimplePartitionWrapperIterator(Partition partition) {this.partition = partition;}
 
     @Override public Partition getCurrent() { return partition; }
+    @Override public String getCatName() { return partition.getCatName(); }
     @Override public String getDbName() { return partition.getDbName(); }
     @Override public String getTableName() { return partition.getTableName(); }
     @Override public Map<String, String> getParameters() { return partition.getParameters(); }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
index 36b05f7..61e00ea 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
@@ -49,6 +49,11 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    partitionSpec.setCatName(catName);
+  }
+
+  @Override
   public void setDbName(String dbName) {
     partitionSpec.setDbName(dbName);
   }
@@ -59,6 +64,11 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return partitionSpec.getCatName();
+  }
+
+  @Override
   public String getDbName() {
     return partitionSpec.getDbName();
   }
@@ -121,7 +131,7 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
       StorageDescriptor partSD = new StorageDescriptor(pSpec.getSd());
       partSD.setLocation(partSD.getLocation() + partWithoutSD.getRelativePath());
 
-      return new Partition(
+      Partition p = new Partition(
           partWithoutSD.getValues(),
           partitionSpecWithSharedSDProxy.partitionSpec.getDbName(),
           partitionSpecWithSharedSDProxy.partitionSpec.getTableName(),
@@ -130,6 +140,13 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
           partSD,
           partWithoutSD.getParameters()
       );
+      p.setCatName(partitionSpecWithSharedSDProxy.partitionSpec.getCatName());
+      return p;
+    }
+
+    @Override
+    public String getCatName() {
+      return partitionSpecWithSharedSDProxy.partitionSpec.getCatName();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
index 7f4d9b0..9cdf271 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
@@ -46,6 +46,8 @@ public class SmokeTest {
   private static final String tableName = "internal_smoke_test_table";
   private static final String partValue = "internal_smoke_test_val1";
 
+  private static Configuration conf;
+
   private SmokeTest() {
 
   }
@@ -63,25 +65,22 @@ public class SmokeTest {
     Database db = new DatabaseBuilder()
         .setName(dbName)
         .setLocation(dbDir.getAbsolutePath())
-        .build();
-    client.createDatabase(db);
+        .create(client, conf);
 
     LOG.info("Going to create table " + tableName);
     Table table = new TableBuilder()
-        .setDbName(db)
+        .inDb(db)
         .setTableName(tableName)
         .addCol("col1", ColumnType.INT_TYPE_NAME)
         .addCol("col2", ColumnType.TIMESTAMP_TYPE_NAME)
         .addPartCol("pcol1", ColumnType.STRING_TYPE_NAME)
-        .build();
-    client.createTable(table);
+        .create(client, conf);
 
     LOG.info("Going to create partition with value " + partValue);
     Partition part = new PartitionBuilder()
-        .fromTable(table)
+        .inTable(table)
         .addValue("val1")
-        .build();
-    client.add_partition(part);
+        .addToTable(client, conf);
 
     LOG.info("Going to list the partitions");
     List<Partition> parts = client.listPartitions(dbName, tableName, (short)-1);
@@ -96,7 +95,7 @@ public class SmokeTest {
 
   public static void main(String[] args) throws Exception {
     SmokeTest test = new SmokeTest();
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     IMetaStoreClient client = new HiveMetaStoreClient(conf);
     test.runTest(client);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index b477ce5..8ea6051 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -118,6 +118,28 @@ public class MetaStoreUtils {
   private static final Charset ENCODING = StandardCharsets.UTF_8;
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreUtils.class);
 
+  // The following two are public for any external users who wish to use them.
+  /**
+   * This character is used to mark a database name as having a catalog name prepended.  This
+   * marker should be placed first in the String to make it easy to determine that this has both
+   * a catalog and a database name.  @ is chosen as it is not used in regular expressions.  This
+   * is only intended for use when making old Thrift calls that do not support catalog names.
+   */
+  public static final char CATALOG_DB_THRIFT_NAME_MARKER = '@';
+
+  /**
+   * This String is used to seaprate the catalog name from the database name.  This should only
+   * be used in Strings that are prepended with {@link #CATALOG_DB_THRIFT_NAME_MARKER}.  # is
+   * chosen because it is not used in regular expressions.  this is only intended for use when
+   * making old Thrift calls that do not support catalog names.
+   */
+  public static final String CATALOG_DB_SEPARATOR = "#";
+
+  /**
+   * Mark a database as being empty (as distinct from null).
+   */
+  public static final String DB_EMPTY_MARKER = "!";
+
   // Right now we only support one special character '/'.
   // More special characters can be added accordingly in the future.
   // NOTE:
@@ -217,7 +239,7 @@ public class MetaStoreUtils {
 
   // Given a list of partStats, this function will give you an aggr stats
   public static List<ColumnStatisticsObj> aggrPartitionStats(List<ColumnStatistics> partStats,
-      String dbName, String tableName, List<String> partNames, List<String> colNames,
+      String catName, String dbName, String tableName, List<String> partNames, List<String> colNames,
       boolean areAllPartsFound, boolean useDensityFunctionForNDVEstimation, double ndvTuner)
       throws MetaException {
     Map<ColumnStatsAggregator, List<ColStatsObjWithSourceInfo>> colStatsMap =
@@ -237,12 +259,12 @@ public class MetaStoreUtils {
               new ArrayList<ColStatsObjWithSourceInfo>());
         }
         colStatsMap.get(aliasToAggregator.get(obj.getColName()))
-            .add(new ColStatsObjWithSourceInfo(obj, dbName, tableName, partName));
+            .add(new ColStatsObjWithSourceInfo(obj, catName, dbName, tableName, partName));
       }
     }
     if (colStatsMap.size() < 1) {
-      LOG.debug("No stats data found for: dbName= {},  tblName= {}, partNames= {}, colNames= {}",
-          dbName, tableName, partNames, colNames);
+      LOG.debug("No stats data found for: tblName= {}, partNames= {}, colNames= {}",
+          Warehouse.getCatalogQualifiedTableName(catName, dbName, tableName), partNames, colNames);
       return new ArrayList<ColumnStatisticsObj>();
     }
     return aggrPartitionStats(colStatsMap, partNames, areAllPartsFound,
@@ -1622,13 +1644,15 @@ public class MetaStoreUtils {
   // ColumnStatisticsObj with info about its db, table, partition (if table is partitioned)
   public static class ColStatsObjWithSourceInfo {
     private final ColumnStatisticsObj colStatsObj;
+    private final String catName;
     private final String dbName;
     private final String tblName;
     private final String partName;
 
-    public ColStatsObjWithSourceInfo(ColumnStatisticsObj colStatsObj, String dbName, String tblName,
+    public ColStatsObjWithSourceInfo(ColumnStatisticsObj colStatsObj, String catName, String dbName, String tblName,
         String partName) {
       this.colStatsObj = colStatsObj;
+      this.catName = catName;
       this.dbName = dbName;
       this.tblName = tblName;
       this.partName = partName;
@@ -1638,6 +1662,10 @@ public class MetaStoreUtils {
       return colStatsObj;
     }
 
+    public String getCatName() {
+      return catName;
+    }
+
     public String getDbName() {
       return dbName;
     }
@@ -1650,4 +1678,109 @@ public class MetaStoreUtils {
       return partName;
     }
   }
+
+  private static boolean hasCatalogName(String dbName) {
+    return dbName != null && dbName.length() > 0 &&
+        dbName.charAt(0) == CATALOG_DB_THRIFT_NAME_MARKER;
+  }
+
+  /**
+   * Given a catalog name and database name cram them together into one string.  This method can
+   * be used if you do not know the catalog name, in which case the default catalog will be
+   * retrieved from the conf object.  The resulting string can be parsed apart again via
+   * {@link #parseDbName(String, Configuration)}.
+   * @param catalogName catalog name, can be null if no known.
+   * @param dbName database name, can be null or empty.
+   * @param conf configuration object, used to determine default catalog if catalogName is null
+   * @return one string that contains both.
+   */
+  public static String prependCatalogToDbName(@Nullable String catalogName, @Nullable String dbName,
+                                              Configuration conf) {
+    if (catalogName == null) catalogName = getDefaultCatalog(conf);
+    StringBuilder buf = new StringBuilder()
+        .append(CATALOG_DB_THRIFT_NAME_MARKER)
+        .append(catalogName)
+        .append(CATALOG_DB_SEPARATOR);
+    if (dbName != null) {
+      if (dbName.isEmpty()) buf.append(DB_EMPTY_MARKER);
+      else buf.append(dbName);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * Given a catalog name and database name, cram them together into one string.  These can be
+   * parsed apart again via {@link #parseDbName(String, Configuration)}.
+   * @param catalogName catalog name.  This cannot be null.  If this might be null use
+   *                    {@link #prependCatalogToDbName(String, String, Configuration)} instead.
+   * @param dbName database name.
+   * @return one string that contains both.
+   */
+  public static String prependNotNullCatToDbName(String catalogName, String dbName) {
+    assert catalogName != null;
+    return prependCatalogToDbName(catalogName, dbName, null);
+  }
+
+  /**
+   * Prepend the default 'hive' catalog onto the database name.
+   * @param dbName database name
+   * @param conf configuration object, used to determine default catalog
+   * @return one string with the 'hive' catalog name prepended.
+   */
+  public static String prependCatalogToDbName(String dbName, Configuration conf) {
+    return prependCatalogToDbName(null, dbName, conf);
+  }
+
+  private final static String[] nullCatalogAndDatabase = {null, null};
+
+  /**
+   * Parse the catalog name out of the database name.  If no catalog name is present then the
+   * default catalog (as set in configuration file) will be assumed.
+   * @param dbName name of the database.  This may or may not contain the catalog name.
+   * @param conf configuration object, used to determine the default catalog if it is not present
+   *            in the database name.
+   * @return an array of two elements, the first being the catalog name, the second the database
+   * name.
+   * @throws MetaException if the name is not either just a database name or a catalog plus
+   * database name with the proper delimiters.
+   */
+  public static String[] parseDbName(String dbName, Configuration conf) throws MetaException {
+    if (dbName == null) return nullCatalogAndDatabase;
+    if (hasCatalogName(dbName)) {
+      if (dbName.endsWith(CATALOG_DB_SEPARATOR)) {
+        // This means the DB name is null
+        return new String[] {dbName.substring(1, dbName.length() - 1), null};
+      } else if (dbName.endsWith(DB_EMPTY_MARKER)) {
+        // This means the DB name is empty
+        return new String[] {dbName.substring(1, dbName.length() - DB_EMPTY_MARKER.length() - 1), ""};
+      }
+      String[] names = dbName.substring(1).split(CATALOG_DB_SEPARATOR, 2);
+      if (names.length != 2) {
+        throw new MetaException(dbName + " is prepended with the catalog marker but does not " +
+            "appear to have a catalog name in it");
+      }
+      return names;
+    } else {
+      return new String[] {getDefaultCatalog(conf), dbName};
+    }
+  }
+
+  /**
+   * Position in the array returned by {@link #parseDbName} that has the catalog name.
+   */
+  public static final int CAT_NAME = 0;
+  /**
+   * Position in the array returned by {@link #parseDbName} that has the database name.
+   */
+  public static final int DB_NAME = 1;
+
+  public static String getDefaultCatalog(Configuration conf) {
+    if (conf == null) {
+      LOG.warn("Configuration is null, so going with default catalog.");
+      return Warehouse.DEFAULT_CATALOG_NAME;
+    }
+    String catName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CATALOG_DEFAULT);
+    if (catName == null || "".equals(catName)) catName = Warehouse.DEFAULT_CATALOG_NAME;
+    return catName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 7612509..8d5ae5d 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -31,9 +31,15 @@
       <datastore-identity>
         <column name="DB_ID"/>
       </datastore-identity>
+      <index name="UniqueDatabase" unique="true">
+        <column name="NAME"/>
+        <column name="CTLG_NAME"/>
+      </index>
       <field name="name">  
         <column name="NAME" length="128" jdbc-type="VARCHAR"/>
-        <index name="UniqueDatabase" unique="true"/>
+      </field>
+      <field name="catalogName">
+        <column name="CTLG_NAME" length="256" jdbc-type="VARCHAR"/>
       </field>
       <field name="description">
         <column name="DESC" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
@@ -61,6 +67,22 @@
       </field>
     </class>
 
+    <class name="MCatalog" identity-type="datastore" table="CTLGS" detachable="true">
+      <datastore-identity>
+        <column name="CTLG_ID"/>
+      </datastore-identity>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR"/>
+        <index name="UniqueCatalog" unique="true"/>
+      </field>
+      <field name="description">
+        <column name="DESC" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
+      <field name="locationUri">
+        <column name="LOCATION_URI" length="4000" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+    </class>
+
     <class name="MFieldSchema" embedded-only="true" table="TYPE_FIELDS" detachable="true">
       <field name="name">
         <column name="FNAME" length="767" jdbc-type="VARCHAR"/>
@@ -191,6 +213,9 @@
       <datastore-identity>
         <column name="MV_CREATION_METADATA_ID"/>
       </datastore-identity>
+      <field name="catalogName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR"/>
+      </field>
       <field name="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
@@ -827,6 +852,9 @@
         <column name="PART_NAME_ID"/>
       </datastore-identity>
       
+      <field name="catalogName">  
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR"/>
+      </field>
       <field name="dbName">  
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
@@ -874,6 +902,9 @@
         <column name="CS_ID"/>
       </datastore-identity>
 
+      <field name ="catName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
       <field name ="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
@@ -938,6 +969,9 @@
         <column name="CS_ID"/>
       </datastore-identity>
 
+      <field name ="catName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
       <field name ="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
@@ -1092,6 +1126,9 @@
       <field name="eventType">
         <column name="EVENT_TYPE" length="32" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
+      <field name="catalogName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
       <field name="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="true"/>
       </field>

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
index de9688d..0003048 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
@@ -15,8 +15,15 @@ CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000
 -- ----------------------------------------------
 -- DDL Statements for tables
 -- ----------------------------------------------
-
-CREATE TABLE "APP"."DBS" ("DB_ID" BIGINT NOT NULL, "DESC" VARCHAR(4000), "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, "NAME" VARCHAR(128), "OWNER_NAME" VARCHAR(128), "OWNER_TYPE" VARCHAR(10));
+CREATE TABLE "APP"."DBS" (
+  "DB_ID" BIGINT NOT NULL,
+  "DESC" VARCHAR(4000),
+  "DB_LOCATION_URI" VARCHAR(4000) NOT NULL,
+  "NAME" VARCHAR(128),
+  "OWNER_NAME" VARCHAR(128),
+  "OWNER_TYPE" VARCHAR(10),
+  "CTLG_NAME" VARCHAR(256) NOT NULL
+);
 
 CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT);
 
@@ -54,7 +61,15 @@ CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME
 
 CREATE TABLE "APP"."PARTITION_PARAMS" ("PART_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000));
 
-CREATE TABLE "APP"."PARTITION_EVENTS" ("PART_NAME_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128), "EVENT_TIME" BIGINT NOT NULL, "EVENT_TYPE" INTEGER NOT NULL, "PARTITION_NAME" VARCHAR(767), "TBL_NAME" VARCHAR(256));
+CREATE TABLE "APP"."PARTITION_EVENTS" (
+    "PART_NAME_ID" BIGINT NOT NULL,
+    "CAT_NAME" VARCHAR(256),
+    "DB_NAME" VARCHAR(128),
+    "EVENT_TIME" BIGINT NOT NULL,
+    "EVENT_TYPE" INTEGER NOT NULL,
+    "PARTITION_NAME" VARCHAR(767),
+    "TBL_NAME" VARCHAR(256)
+);
 
 CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "COLUMN_NAME" VARCHAR(128) NOT NULL, "TYPE_NAME" VARCHAR(4000) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 
@@ -70,7 +85,29 @@ CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000),
 
 CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL);
 
-CREATE TABLE "APP"."TAB_COL_STATS"("DB_NAME" VARCHAR(128) NOT NULL,"TABLE_NAME" VARCHAR(256) NOT NULL, "COLUMN_NAME" VARCHAR(767) NOT NULL, "COLUMN_TYPE" VARCHAR(128) NOT NULL, "LONG_LOW_VALUE" BIGINT, "LONG_HIGH_VALUE" BIGINT, "DOUBLE_LOW_VALUE" DOUBLE, "DOUBLE_HIGH_VALUE" DOUBLE, "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),"BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),"NUM_DISTINCTS" BIGINT, "NUM_NULLS" BIGINT NOT NULL, "AVG_COL_LEN" DOUBLE, "MAX_COL_LEN" BIGINT, "NUM_TRUES" BIGINT, "NUM_FALSES" BIGINT, "LAST_ANALYZED" BIGINT, "CS_ID" BIGINT NOT NULL, "TBL_ID" BIGINT NOT NULL, "BIT_VECTOR" BLOB);
+CREATE TABLE "APP"."TAB_COL_STATS"(
+    "CAT_NAME" VARCHAR(256) NOT NULL,
+    "DB_NAME" VARCHAR(128) NOT NULL,
+    "TABLE_NAME" VARCHAR(256) NOT NULL,
+    "COLUMN_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_TYPE" VARCHAR(128) NOT NULL,
+    "LONG_LOW_VALUE" BIGINT,
+    "LONG_HIGH_VALUE" BIGINT,
+    "DOUBLE_LOW_VALUE" DOUBLE,
+    "DOUBLE_HIGH_VALUE" DOUBLE,
+    "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),
+    "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),
+    "NUM_DISTINCTS" BIGINT,
+    "NUM_NULLS" BIGINT NOT NULL,
+    "AVG_COL_LEN" DOUBLE,
+    "MAX_COL_LEN" BIGINT,
+    "NUM_TRUES" BIGINT,
+    "NUM_FALSES" BIGINT,
+    "LAST_ANALYZED" BIGINT,
+    "CS_ID" BIGINT NOT NULL,
+    "TBL_ID" BIGINT NOT NULL,
+    "BIT_VECTOR" BLOB
+);
 
 CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB);
 
@@ -96,7 +133,30 @@ CREATE TABLE "APP"."MASTER_KEYS" ("KEY_ID" INTEGER NOT NULL generated always as
 
 CREATE TABLE "APP"."DELEGATION_TOKENS" ( "TOKEN_IDENT" VARCHAR(767) NOT NULL, "TOKEN" VARCHAR(767));
 
-CREATE TABLE "APP"."PART_COL_STATS"("DB_NAME" VARCHAR(128) NOT NULL,"TABLE_NAME" VARCHAR(256) NOT NULL, "PARTITION_NAME" VARCHAR(767) NOT NULL, "COLUMN_NAME" VARCHAR(767) NOT NULL, "COLUMN_TYPE" VARCHAR(128) NOT NULL, "LONG_LOW_VALUE" BIGINT, "LONG_HIGH_VALUE" BIGINT, "DOUBLE_LOW_VALUE" DOUBLE, "DOUBLE_HIGH_VALUE" DOUBLE, "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),"NUM_DISTINCTS" BIGINT, "BIT_VECTOR" BLOB, "NUM_NULLS" BIGINT NOT NULL, "AVG_COL_LEN" DOUBLE, "MAX_COL_LEN" BIGINT, "NUM_TRUES" BIGINT, "NUM_FALSES" BIGINT, "LAST_ANALYZED" BIGINT, "CS_ID" BIGINT NOT NULL, "PART_ID" BIGINT NOT NULL);
+CREATE TABLE "APP"."PART_COL_STATS"(
+    "CAT_NAME" VARCHAR(256) NOT NULL,
+    "DB_NAME" VARCHAR(128) NOT NULL,
+    "TABLE_NAME" VARCHAR(256) NOT NULL,
+    "PARTITION_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_TYPE" VARCHAR(128) NOT NULL,
+    "LONG_LOW_VALUE" BIGINT,
+    "LONG_HIGH_VALUE" BIGINT,
+    "DOUBLE_LOW_VALUE" DOUBLE,
+    "DOUBLE_HIGH_VALUE" DOUBLE,
+    "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),
+    "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),
+    "NUM_DISTINCTS" BIGINT,
+    "BIT_VECTOR" BLOB,
+    "NUM_NULLS" BIGINT NOT NULL,
+    "AVG_COL_LEN" DOUBLE,
+    "MAX_COL_LEN" BIGINT,
+    "NUM_TRUES" BIGINT,
+    "NUM_FALSES" BIGINT,
+    "LAST_ANALYZED" BIGINT,
+    "CS_ID" BIGINT NOT NULL,
+    "PART_ID" BIGINT NOT NULL
+);
 
 CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255));
 
@@ -104,7 +164,17 @@ CREATE TABLE "APP"."FUNCS" ("FUNC_ID" BIGINT NOT NULL, "CLASS_NAME" VARCHAR(4000
 
 CREATE TABLE "APP"."FUNC_RU" ("FUNC_ID" BIGINT NOT NULL, "RESOURCE_TYPE" INTEGER NOT NULL, "RESOURCE_URI" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL);
 
-CREATE TABLE "APP"."NOTIFICATION_LOG" ("NL_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128), "EVENT_ID" BIGINT NOT NULL, "EVENT_TIME" INTEGER NOT NULL, "EVENT_TYPE" VARCHAR(32) NOT NULL, "MESSAGE" CLOB, "TBL_NAME" VARCHAR(256), "MESSAGE_FORMAT" VARCHAR(16));
+CREATE TABLE "APP"."NOTIFICATION_LOG" (
+    "NL_ID" BIGINT NOT NULL,
+    "CAT_NAME" VARCHAR(256),
+    "DB_NAME" VARCHAR(128),
+    "EVENT_ID" BIGINT NOT NULL,
+    "EVENT_TIME" INTEGER NOT NULL,
+    "EVENT_TYPE" VARCHAR(32) NOT NULL,
+    "MESSAGE" CLOB,
+    "TBL_NAME" VARCHAR(256),
+    "MESSAGE_FORMAT" VARCHAR(16)
+);
 
 CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL);
 
@@ -124,6 +194,7 @@ CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NU
 
 CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "MV_CREATION_METADATA_ID" BIGINT NOT NULL,
+  "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
   "TXN_LIST" CLOB
@@ -134,6 +205,12 @@ CREATE TABLE "APP"."MV_TABLES_USED" (
   "TBL_ID" BIGINT NOT NULL
 );
 
+CREATE TABLE "APP"."CTLGS" (
+    "CTLG_ID" BIGINT NOT NULL,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL);
+
 -- ----------------------------------------------
 -- DML Statements
 -- ----------------------------------------------
@@ -150,7 +227,7 @@ CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("TBL_ID
 
 CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE");
 
-CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
 
 CREATE INDEX "APP"."PARTPRIVILEGEINDEX" ON "APP"."PART_PRIVS" ("PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE");
 
@@ -160,7 +237,7 @@ CREATE INDEX "APP"."TABLEPRIVILEGEINDEX" ON "APP"."TBL_PRIVS" ("TBL_ID", "PRINCI
 
 CREATE UNIQUE INDEX "APP"."UNIQUETABLE" ON "APP"."TBLS" ("TBL_NAME", "DB_ID");
 
-CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME");
+CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME");
 
 CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE");
 
@@ -192,6 +269,9 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "E
 
 CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME");
 
+CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME");
+
+
 -- ----------------------------------------------
 -- DDL Statements for keys
 -- ----------------------------------------------
@@ -289,6 +369,9 @@ ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD CONSTRAINT "PROPERTY_KEY_PK" PRI
 
 ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
 
+ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID");
+
+
 -- foreign
 ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
@@ -322,6 +405,8 @@ ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFEREN
 
 ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
 ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
 ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
@@ -394,6 +479,8 @@ ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN K
 
 ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
 -- ----------------------------------------------
 -- DDL Statements for checks
 -- ----------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 8925fa8..6aa2e82 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -81,6 +81,7 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SC
 -- create mv_creation_metadata table
 CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "MV_CREATION_METADATA_ID" BIGINT NOT NULL,
+  "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
   "TXN_LIST" CLOB
@@ -160,5 +161,62 @@ ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD COLUMN "DEFAULT_VALUE" VARCHAR(400);
 
 ALTER TABLE "APP"."HIVE_LOCKS" ALTER COLUMN "HL_TXNID" NOT NULL;
 
+-- Create new Catalog table
+-- HIVE-18755, add catalogs
+-- new catalogs table
+CREATE TABLE "APP"."CTLGS" (
+    "CTLG_ID" BIGINT NOT NULL,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL);
+
+ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLGS_PK" PRIMARY KEY ("CTLG_ID");
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO "APP"."CTLGS" VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX "APP"."UNIQUE_DATABASE";
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE "APP"."DBS" ADD COLUMN "CTLG_NAME" VARCHAR(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE "APP"."DBS" 
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."DBS" ALTER COLUMN "CTLG_NAME" NOT NULL;
+
+-- Put back the unique index 
+CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
+-- Add columns to table stats and part stats
+ALTER TABLE "APP"."TAB_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+ALTER TABLE "APP"."PART_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Set the existing column names to Hive
+UPDATE "APP"."TAB_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+UPDATE "APP"."PART_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."TAB_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+ALTER TABLE "APP"."PART_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX "APP"."PCS_STATS_IDX";
+CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+
+-- Add column to partition events
+ALTER TABLE "APP"."PARTITION_EVENTS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Add column to notification log
+ALTER TABLE "APP"."NOTIFICATION_LOG" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
index 68237ec..77afd60 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
@@ -93,12 +93,13 @@ CREATE TABLE PART_COL_STATS
     NUM_TRUES bigint NULL,
     PART_ID bigint NULL,
     PARTITION_NAME nvarchar(767) NOT NULL,
-    "TABLE_NAME" nvarchar(256) NOT NULL
+    "TABLE_NAME" nvarchar(256) NOT NULL,
+    "CAT_NAME" nvarchar(256) NOT NULL
 );
 
 ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_PK PRIMARY KEY (CS_ID);
 
-CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
 
 -- Table PART_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
 CREATE TABLE PART_PRIVS
@@ -236,7 +237,8 @@ CREATE TABLE TAB_COL_STATS
     NUM_NULLS bigint NOT NULL,
     NUM_TRUES bigint NULL,
     TBL_ID bigint NULL,
-    "TABLE_NAME" nvarchar(256) NOT NULL
+    "TABLE_NAME" nvarchar(256) NOT NULL,
+    "CAT_NAME" nvarchar(256) NOT NULL
 );
 
 ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PK PRIMARY KEY (CS_ID);
@@ -276,7 +278,8 @@ CREATE TABLE DBS
     DB_LOCATION_URI nvarchar(4000) NOT NULL,
     "NAME" nvarchar(128) NULL,
     OWNER_NAME nvarchar(128) NULL,
-    OWNER_TYPE nvarchar(10) NULL
+    OWNER_TYPE nvarchar(10) NULL,
+    CTLG_NAME nvarchar(256)
 );
 
 ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
@@ -374,6 +377,7 @@ ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
     TXN_LIST text NULL
@@ -382,6 +386,7 @@ CREATE TABLE MV_CREATION_METADATA
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
 CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME,DB_NAME);
 
+
 CREATE TABLE MV_TABLES_USED
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
@@ -411,6 +416,7 @@ ALTER TABLE SDS ADD CONSTRAINT SDS_PK PRIMARY KEY (SD_ID);
 CREATE TABLE PARTITION_EVENTS
 (
     PART_NAME_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NULL,
     DB_NAME nvarchar(128) NULL,
     EVENT_TIME bigint NOT NULL,
     EVENT_TYPE int NOT NULL,
@@ -604,6 +610,7 @@ CREATE TABLE NOTIFICATION_LOG
     EVENT_ID bigint NOT NULL,
     EVENT_TIME int NOT NULL,
     EVENT_TYPE nvarchar(32) NOT NULL,
+    CAT_NAME nvarchar(128) NULL,
     DB_NAME nvarchar(128) NULL,
     TBL_NAME nvarchar(256) NULL,
     MESSAGE_FORMAT nvarchar(16),
@@ -677,6 +684,15 @@ CREATE TABLE WM_MAPPING
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_PK PRIMARY KEY (MAPPING_ID);
 
+CREATE TABLE CTLGS (
+      CTLG_ID bigint primary key,
+      "NAME" nvarchar(256),
+      "DESC" nvarchar(4000),
+      LOCATION_URI nvarchar(4000) not null
+);
+
+CREATE UNIQUE INDEX UNIQUE_CTLG ON CTLGS ("NAME");
+
 -- Constraints for table MASTER_KEYS for class(es) [org.apache.hadoop.hive.metastore.model.MMasterKey]
 
 -- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]
@@ -770,7 +786,7 @@ CREATE INDEX TABLEPRIVILEGEINDEX ON TBL_PRIVS (TBL_ID,PRINCIPAL_NAME,PRINCIPAL_T
 
 
 -- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
-CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME");
+CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME", "CTLG_NAME");
 
 
 -- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
@@ -958,6 +974,7 @@ CREATE UNIQUE INDEX UNIQUE_WM_MAPPING ON WM_MAPPING (RP_ID, ENTITY_TYPE, ENTITY_
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK1 FOREIGN KEY (RP_ID) REFERENCES WM_RESOURCEPLAN (RP_ID);
 
+ALTER TABLE DBS ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES CTLGS ("NAME");
 -- -----------------------------------------------------------------------------------------------------------------------------------------------
 -- Transaction and Lock Tables
 -- These are not part of package jdo, so if you are going to regenerate this file you need to manually add the following section back to the file.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index c5041b3..b780329 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -140,6 +140,7 @@ CREATE TABLE "SCHEMA_VERSION" (
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
     TXN_LIST text NULL
@@ -212,6 +213,64 @@ ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
 
 ALTER TABLE HIVE_LOCKS ALTER COLUMN HL_TXNID bigint NOT NULL;
 
+-- HIVE-18755, add catalogs
+-- new catalog table
+CREATE TABLE CTLGS (
+      CTLG_ID bigint primary key,
+      "NAME" nvarchar(256),
+      "DESC" nvarchar(4000),
+      LOCATION_URI nvarchar(4000) not null
+);
+
+-- Create unique index on CTLGS.NAME
+CREATE UNIQUE INDEX UNIQUE_CTLG ON CTLGS ("NAME");
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO CTLGS VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX UNIQUEDATABASE ON DBS;
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE DBS ADD CTLG_NAME nvarchar(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE DBS
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE DBS ALTER COLUMN CTLG_NAME nvarchar(256) NOT NULL;
+
+-- Put back the unique index
+CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE DBS ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES CTLGS ("NAME");
+
+-- Add columns to table stats and part stats
+ALTER TABLE TAB_COL_STATS ADD CAT_NAME nvarchar(256);
+ALTER TABLE PART_COL_STATS ADD CAT_NAME nvarchar(256);
+
+-- Set the existing column names to Hive
+UPDATE TAB_COL_STATS
+  SET CAT_NAME = 'hive';
+UPDATE PART_COL_STATS
+  SET CAT_NAME = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE TAB_COL_STATS ALTER COLUMN CAT_NAME nvarchar(256) NOT NULL;
+ALTER TABLE PART_COL_STATS ALTER COLUMN CAT_NAME nvarchar(256) NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX PCS_STATS_IDX ON PART_COL_STATS;
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME, PARTITION_NAME);
+
+-- Add columns to partition events
+ALTER TABLE PARTITION_EVENTS ADD CAT_NAME nvarchar(256);
+
+-- Add columns to notification log
+ALTER TABLE NOTIFICATION_LOG ADD CAT_NAME nvarchar(256);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index 3e2db2a..adbe129 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -77,6 +77,15 @@ CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
 
+CREATE TABLE `CTLGS` (
+    `CTLG_ID` BIGINT PRIMARY KEY,
+    `NAME` VARCHAR(256),
+    `DESC` VARCHAR(4000),
+    `LOCATION_URI` VARCHAR(4000) NOT NULL,
+    UNIQUE KEY `UNIQUE_CATALOG` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+
 --
 -- Table structure for table `DBS`
 --
@@ -90,8 +99,10 @@ CREATE TABLE IF NOT EXISTS `DBS` (
   `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `OWNER_TYPE` varchar(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CTLG_NAME` varchar(256) NOT NULL,
   PRIMARY KEY (`DB_ID`),
-  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`)
+  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`, `CTLG_NAME`),
+  CONSTRAINT `CTLG_FK1` FOREIGN KEY (`CTLG_NAME`) REFERENCES `CTLGS` (`NAME`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
 
@@ -228,6 +239,7 @@ CREATE TABLE IF NOT EXISTS `PARTITIONS` (
 /*!40101 SET character_set_client = utf8 */;
 CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` (
   `PART_NAME_ID` bigint(20) NOT NULL,
+  `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `EVENT_TIME` bigint(20) NOT NULL,
   `EVENT_TYPE` int(11) NOT NULL,
@@ -581,6 +593,7 @@ CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
 /*!40101 SET character_set_client = utf8 */;
 CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
   `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TXN_LIST` TEXT DEFAULT NULL,
@@ -684,6 +697,7 @@ CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
 --
 CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
  `CS_ID` bigint(20) NOT NULL,
+ `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `TABLE_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
@@ -712,6 +726,7 @@ CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
 --
 CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
  `CS_ID` bigint(20) NOT NULL,
+ `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `TABLE_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
@@ -736,7 +751,7 @@ CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
   CONSTRAINT `PART_COL_STATS_FK` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME) USING BTREE;
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME) USING BTREE;
 
 --
 -- Table structure for table `TYPES`
@@ -833,6 +848,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_LOG`
     `EVENT_ID` BIGINT(20) NOT NULL,
     `EVENT_TIME` INT(11) NOT NULL,
     `EVENT_TYPE` varchar(32) NOT NULL,
+    `CAT_NAME` varchar(256),
     `DB_NAME` varchar(128),
     `TBL_NAME` varchar(256),
     `MESSAGE` longtext,