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/12 23:55:43 UTC

[03/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
new file mode 100644
index 0000000..ceb0f49
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
@@ -0,0 +1,108 @@
+/*
+ * 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.client.builder;
+
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+
+public class SchemaVersionBuilder extends SerdeAndColsBuilder<SchemaVersionBuilder> {
+  private String schemaName, dbName; // required
+  private int version; // required
+  private long createdAt; // required
+  private SchemaVersionState state; // optional
+  private String description; // optional
+  private String schemaText; // optional
+  private String fingerprint; // optional
+  private String name; // optional
+
+  public SchemaVersionBuilder() {
+    createdAt = System.currentTimeMillis() / 1000;
+    version = -1;
+    super.setChild(this);
+  }
+
+  public SchemaVersionBuilder setSchemaName(String schemaName) {
+    this.schemaName = schemaName;
+    return this;
+  }
+
+  public SchemaVersionBuilder setDbName(String dbName) {
+    this.dbName = dbName;
+    return this;
+  }
+
+  public SchemaVersionBuilder versionOf(ISchema schema) {
+    this.dbName = schema.getDbName();
+    this.schemaName = schema.getName();
+    return this;
+  }
+
+  public SchemaVersionBuilder setVersion(int version) {
+    this.version = version;
+    return this;
+  }
+
+  public SchemaVersionBuilder setCreatedAt(long createdAt) {
+    this.createdAt = createdAt;
+    return this;
+  }
+
+  public SchemaVersionBuilder setState(
+      SchemaVersionState state) {
+    this.state = state;
+    return this;
+  }
+
+  public SchemaVersionBuilder setDescription(String description) {
+    this.description = description;
+    return this;
+  }
+
+  public SchemaVersionBuilder setSchemaText(String schemaText) {
+    this.schemaText = schemaText;
+    return this;
+  }
+
+  public SchemaVersionBuilder setFingerprint(String fingerprint) {
+    this.fingerprint = fingerprint;
+    return this;
+  }
+
+  public SchemaVersionBuilder setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public SchemaVersion build() throws MetaException {
+    if (schemaName == null || dbName == null || version < 0) {
+      throw new MetaException("You must provide the database name, schema name, and schema version");
+    }
+    SchemaVersion schemaVersion =
+        new SchemaVersion(new ISchemaName(dbName, schemaName), version, createdAt, getCols());
+    if (state != null) schemaVersion.setState(state);
+    if (description != null) schemaVersion.setDescription(description);
+    if (schemaText != null) schemaVersion.setSchemaText(schemaText);
+    if (fingerprint != null) schemaVersion.setFingerprint(fingerprint);
+    if (name != null) schemaVersion.setName(name);
+    schemaVersion.setSerDe(buildSerde());
+    return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
new file mode 100644
index 0000000..e1405d3
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
@@ -0,0 +1,124 @@
+/*
+ * 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.client.builder;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SerdeType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This collects together SerdeInfo and columns, since StorageDescriptor and SchemaVersion share
+ * those traits.
+ * @param <T>
+ */
+abstract class SerdeAndColsBuilder<T> {
+  private static final String SERDE_LIB = "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe";
+
+  private List<FieldSchema> cols;
+  private String serdeName, serdeLib, serdeDescription, serdeSerializerClass, serdeDeserializerClass;
+  private Map<String, String> serdeParams;
+  private SerdeType serdeType;
+  protected T child;
+
+  protected SerdeAndColsBuilder() {
+    serdeParams = new HashMap<>();
+    serdeLib = SERDE_LIB;
+  }
+
+  protected void setChild(T child) {
+    this.child = child;
+  }
+
+  protected SerDeInfo buildSerde() {
+    SerDeInfo serDeInfo = new SerDeInfo(serdeName, serdeLib, serdeParams);
+    if (serdeDescription != null) serDeInfo.setDescription(serdeDescription);
+    if (serdeSerializerClass != null) serDeInfo.setSerializerClass(serdeSerializerClass);
+    if (serdeDeserializerClass != null) serDeInfo.setDeserializerClass(serdeDeserializerClass);
+    if (serdeType != null) serDeInfo.setSerdeType(serdeType);
+    return serDeInfo;
+  }
+
+  protected List<FieldSchema> getCols() throws MetaException {
+    if (cols == null) throw new MetaException("You must provide the columns");
+    return cols;
+  }
+
+  public T setCols(
+      List<FieldSchema> cols) {
+    this.cols = cols;
+    return child;
+  }
+
+  public T addCol(String name, String type, String comment) {
+    if (cols == null) cols = new ArrayList<>();
+    cols.add(new FieldSchema(name, type, comment));
+    return child;
+  }
+
+  public T addCol(String name, String type) {
+    return addCol(name, type, "");
+  }
+
+  public T setSerdeName(String serdeName) {
+    this.serdeName = serdeName;
+    return child;
+  }
+
+  public T setSerdeLib(String serdeLib) {
+    this.serdeLib = serdeLib;
+    return child;
+  }
+
+  public T setSerdeDescription(String serdeDescription) {
+    this.serdeDescription = serdeDescription;
+    return child;
+  }
+
+  public T setSerdeSerializerClass(String serdeSerializerClass) {
+    this.serdeSerializerClass = serdeSerializerClass;
+    return child;
+  }
+
+  public T setSerdeDeserializerClass(String serdeDeserializerClass) {
+    this.serdeDeserializerClass = serdeDeserializerClass;
+    return child;
+  }
+
+  public T setSerdeParams(
+      Map<String, String> serdeParams) {
+    this.serdeParams = serdeParams;
+    return child;
+  }
+
+  public T addSerdeParam(String key, String value) {
+    if (serdeParams == null) serdeParams = new HashMap<>();
+    serdeParams.put(key, value);
+    return child;
+  }
+
+  public T setSerdeType(SerdeType serdeType) {
+    this.serdeType = serdeType;
+    return child;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/StorageDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/StorageDescriptorBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/StorageDescriptorBuilder.java
index 39d1fa2..433e7c7 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/StorageDescriptorBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/StorageDescriptorBuilder.java
@@ -34,44 +34,36 @@ import java.util.Map;
  * defaults for everything else.  This is intended for use just by objects that have a StorageDescriptor,
  * not direct use.
  */
-abstract class StorageDescriptorBuilder<T> {
-  private static final String SERDE_LIB = "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe";
+abstract class StorageDescriptorBuilder<T> extends SerdeAndColsBuilder<T> {
   private static final String INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.HiveInputFormat";
   private static final String OUTPUT_FORMAT = "org.apache.hadoop.hive.ql.io.HiveOutputFormat";
 
-  private String location, inputFormat, outputFormat, serdeName, serdeLib;
-  private List<FieldSchema> cols;
+  private String location, inputFormat, outputFormat;
   private int numBuckets;
-  private Map<String, String> storageDescriptorParams, serdeParams;
+  private Map<String, String> storageDescriptorParams;
   private boolean compressed, storedAsSubDirectories;
   private List<String> bucketCols, skewedColNames;
   private List<Order> sortCols;
   private List<List<String>> skewedColValues;
   private Map<List<String>, String> skewedColValueLocationMaps;
-  // This enables us to return the correct type from the builder
-  private T child;
 
   protected StorageDescriptorBuilder() {
     // Set some reasonable defaults
     storageDescriptorParams = new HashMap<>();
-    serdeParams = new HashMap<>();
     bucketCols = new ArrayList<>();
     sortCols = new ArrayList<>();
     numBuckets = 0;
     compressed = false;
     inputFormat = INPUT_FORMAT;
     outputFormat = OUTPUT_FORMAT;
-    serdeLib = SERDE_LIB;
     skewedColNames = new ArrayList<>();
     skewedColValues = new ArrayList<>();
     skewedColValueLocationMaps = new HashMap<>();
   }
 
   protected StorageDescriptor buildSd() throws MetaException {
-    if (cols == null) throw new MetaException("You must provide the columns");
-    SerDeInfo serdeInfo = new SerDeInfo(serdeName, serdeLib, serdeParams);
-    StorageDescriptor sd = new StorageDescriptor(cols, location, inputFormat, outputFormat,
-        compressed, numBuckets, serdeInfo, bucketCols, sortCols, storageDescriptorParams);
+    StorageDescriptor sd = new StorageDescriptor(getCols(), location, inputFormat, outputFormat,
+        compressed, numBuckets, buildSerde(), bucketCols, sortCols, storageDescriptorParams);
     sd.setStoredAsSubDirectories(storedAsSubDirectories);
     if (skewedColNames != null) {
       SkewedInfo skewed = new SkewedInfo(skewedColNames, skewedColValues,
@@ -81,10 +73,6 @@ abstract class StorageDescriptorBuilder<T> {
     return sd;
   }
 
-  protected void setChild(T child) {
-    this.child = child;
-  }
-
   public T setLocation(String location) {
     this.location = location;
     return child;
@@ -100,30 +88,6 @@ abstract class StorageDescriptorBuilder<T> {
     return child;
   }
 
-  public T setSerdeName(String serdeName) {
-    this.serdeName = serdeName;
-    return child;
-  }
-
-  public T setSerdeLib(String serdeLib) {
-    this.serdeLib = serdeLib;
-    return child;
-  }
-  public T setCols(List<FieldSchema> cols) {
-    this.cols = cols;
-    return child;
-  }
-
-  public T addCol(String name, String type, String comment) {
-    if (cols == null) cols = new ArrayList<>();
-    cols.add(new FieldSchema(name, type, comment));
-    return child;
-  }
-
-  public T addCol(String name, String type) {
-    return addCol(name, type, "");
-  }
-
   public T setNumBuckets(int numBuckets) {
     this.numBuckets = numBuckets;
     return child;
@@ -141,17 +105,6 @@ abstract class StorageDescriptorBuilder<T> {
     return child;
   }
 
-  public T setSerdeParams(Map<String, String> serdeParams) {
-    this.serdeParams = serdeParams;
-    return child;
-  }
-
-  public T addSerdeParam(String key, String value) {
-    if (serdeParams == null) serdeParams = new HashMap<>();
-    serdeParams.put(key, value);
-    return child;
-  }
-
   public T setCompressed(boolean compressed) {
     this.compressed = compressed;
     return child;

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddSchemaVersionEvent.java
new file mode 100644
index 0000000..e6839de
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddSchemaVersionEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddSchemaVersionEvent extends ListenerEvent {
+
+  private final SchemaVersion schemaVersion;
+
+  public AddSchemaVersionEvent(boolean status, IHMSHandler handler,
+                               SchemaVersion schemaVersion) {
+    super(status, handler);
+    this.schemaVersion = schemaVersion;
+  }
+
+  public SchemaVersion getSchemaVersion() {
+    return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterISchemaEvent.java
new file mode 100644
index 0000000..eaf1db5
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterISchemaEvent.java
@@ -0,0 +1,45 @@
+/*
+ * 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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AlterISchemaEvent extends ListenerEvent {
+
+  private final ISchema oldSchema, newSchema;
+
+  public AlterISchemaEvent(boolean status, IHMSHandler handler,
+                           ISchema oldSchema, ISchema newSchema) {
+    super(status, handler);
+    this.oldSchema = oldSchema;
+    this.newSchema = newSchema;
+  }
+
+  public ISchema getOldSchema() {
+    return oldSchema;
+  }
+
+  public ISchema getNewSchema() {
+    return newSchema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterSchemaVersionEvent.java
new file mode 100644
index 0000000..76b834e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterSchemaVersionEvent.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AlterSchemaVersionEvent extends ListenerEvent {
+
+  private final SchemaVersion oldSchemaVersion, newSchemaVersion;
+
+  public AlterSchemaVersionEvent(boolean status, IHMSHandler handler,
+                                 SchemaVersion oldSchemaVersion,
+                                 SchemaVersion newSchemaVersion) {
+    super(status, handler);
+    this.oldSchemaVersion = oldSchemaVersion;
+    this.newSchemaVersion = newSchemaVersion;
+  }
+
+  public SchemaVersion getOldSchemaVersion() {
+    return oldSchemaVersion;
+  }
+
+  public SchemaVersion getNewSchemaVersion() {
+    return newSchemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateISchemaEvent.java
new file mode 100644
index 0000000..348f8d3
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateISchemaEvent.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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CreateISchemaEvent extends ListenerEvent {
+
+  private final ISchema schema;
+
+  public CreateISchemaEvent(boolean status, IHMSHandler handler, ISchema schema) {
+    super(status, handler);
+    this.schema = schema;
+  }
+
+  public ISchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropISchemaEvent.java
new file mode 100644
index 0000000..7c03638
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropISchemaEvent.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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropISchemaEvent extends ListenerEvent {
+
+  private final ISchema schema;
+
+  public DropISchemaEvent(boolean status, IHMSHandler handler, ISchema schema) {
+    super(status, handler);
+    this.schema = schema;
+  }
+
+  public ISchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropSchemaVersionEvent.java
new file mode 100644
index 0000000..c722c33
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropSchemaVersionEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropSchemaVersionEvent extends ListenerEvent {
+
+  private final SchemaVersion schemaVersion;
+
+  public DropSchemaVersionEvent(boolean status, IHMSHandler handler,
+                                SchemaVersion schemaVersion) {
+    super(status, handler);
+    this.schemaVersion = schemaVersion;
+  }
+
+  public SchemaVersion getSchemaVersion() {
+    return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddSchemaVersionEvent.java
new file mode 100644
index 0000000..fc345f5
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddSchemaVersionEvent.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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAddSchemaVersionEvent extends PreEventContext {
+
+  private final SchemaVersion schemaVersion;
+
+  public PreAddSchemaVersionEvent(IHMSHandler handler, SchemaVersion schemaVersion) {
+    super(PreEventType.ADD_SCHEMA_VERSION, handler);
+    this.schemaVersion = schemaVersion;
+  }
+
+  public SchemaVersion getSchemaVersion() {
+    return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterISchemaEvent.java
new file mode 100644
index 0000000..3df3780
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterISchemaEvent.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAlterISchemaEvent extends PreEventContext {
+
+  private final ISchema oldSchema, newSchema;
+
+  public PreAlterISchemaEvent(IHMSHandler handler, ISchema oldSchema, ISchema newSchema) {
+    super(PreEventType.ALTER_ISCHEMA, handler);
+    this.oldSchema = oldSchema;
+    this.newSchema = newSchema;
+  }
+
+  public ISchema getOldSchema() {
+    return oldSchema;
+  }
+
+  public ISchema getNewSchema() {
+    return newSchema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterSchemaVersionEvent.java
new file mode 100644
index 0000000..63ddb3b
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterSchemaVersionEvent.java
@@ -0,0 +1,45 @@
+/*
+ * 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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAlterSchemaVersionEvent extends PreEventContext {
+
+  private final SchemaVersion oldSchemaVersion, newSchemaVersion;
+
+  public PreAlterSchemaVersionEvent(IHMSHandler handler, SchemaVersion oldSchemaVersion,
+                                    SchemaVersion newSchemaVersion) {
+    super(PreEventType.ALTER_SCHEMA_VERSION, handler);
+    this.oldSchemaVersion = oldSchemaVersion;
+    this.newSchemaVersion = newSchemaVersion;
+  }
+
+  public SchemaVersion getOldSchemaVersion() {
+    return oldSchemaVersion;
+  }
+
+  public SchemaVersion getNewSchemaVersion() {
+    return newSchemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateISchemaEvent.java
new file mode 100644
index 0000000..d8e9f04
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateISchemaEvent.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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateISchemaEvent extends PreEventContext {
+
+  private final ISchema schema;
+
+  public PreCreateISchemaEvent(IHMSHandler handler, ISchema schema) {
+    super(PreEventType.CREATE_ISCHEMA, handler);
+    this.schema = schema;
+  }
+
+  public ISchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropISchemaEvent.java
new file mode 100644
index 0000000..5755374
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropISchemaEvent.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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropISchemaEvent extends PreEventContext {
+
+  private final ISchema schema;
+
+  public PreDropISchemaEvent(IHMSHandler handler, ISchema schema) {
+    super(PreEventType.DROP_ISCHEMA, handler);
+    this.schema = schema;
+  }
+
+  public ISchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropSchemaVersionEvent.java
new file mode 100644
index 0000000..2958bd9
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropSchemaVersionEvent.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.SchemaVersion;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropSchemaVersionEvent extends PreEventContext {
+
+  private final SchemaVersion schemaVersion;
+
+  public PreDropSchemaVersionEvent(IHMSHandler handler, SchemaVersion schemaVersion) {
+    super(PreEventType.DROP_SCHEMA_VERSION, handler);
+    this.schemaVersion = schemaVersion;
+  }
+
+  public SchemaVersion getSchemaVersion() {
+    return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 c5efde3..7ddb8fe 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
@@ -42,7 +42,15 @@ public abstract class PreEventContext {
     AUTHORIZATION_API_CALL,
     READ_TABLE,
     READ_DATABASE,
-    ALTER_DATABASE
+    ALTER_DATABASE,
+    CREATE_ISCHEMA,
+    ALTER_ISCHEMA,
+    DROP_ISCHEMA,
+    ADD_SCHEMA_VERSION,
+    ALTER_SCHEMA_VERSION,
+    DROP_SCHEMA_VERSION,
+    READ_ISCHEMA,
+    READ_SCHEMA_VERSION
   }
 
   private final PreEventType eventType;

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadISchemaEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadISchemaEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadISchemaEvent.java
new file mode 100644
index 0000000..de8ce04
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadISchemaEvent.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.ISchema;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreReadISchemaEvent extends PreEventContext {
+
+  private final ISchema schema;
+
+  public PreReadISchemaEvent(IHMSHandler handler, ISchema schema) {
+    super(PreEventType.READ_ISCHEMA, handler);
+    this.schema = schema;
+  }
+
+  public ISchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadhSchemaVersionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadhSchemaVersionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadhSchemaVersionEvent.java
new file mode 100644
index 0000000..fbe4879
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadhSchemaVersionEvent.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+
+import java.util.List;
+
+public class PreReadhSchemaVersionEvent extends PreEventContext {
+  private final List<SchemaVersion> schemaVersions;
+
+  public PreReadhSchemaVersionEvent(IHMSHandler handler, List<SchemaVersion> schemaVersions) {
+    super(PreEventType.READ_SCHEMA_VERSION, handler);
+    this.schemaVersions = schemaVersions;
+  }
+
+  public List<SchemaVersion> getSchemaVersions() {
+    return schemaVersions;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 92d2eb4..8578d4a 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
@@ -47,7 +47,13 @@ public abstract class EventMessage {
     ADD_FOREIGNKEY(MessageFactory.ADD_FOREIGNKEY_EVENT),
     ADD_UNIQUECONSTRAINT(MessageFactory.ADD_UNIQUECONSTRAINT_EVENT),
     ADD_NOTNULLCONSTRAINT(MessageFactory.ADD_NOTNULLCONSTRAINT_EVENT),
-    DROP_CONSTRAINT(MessageFactory.DROP_CONSTRAINT_EVENT);
+    DROP_CONSTRAINT(MessageFactory.DROP_CONSTRAINT_EVENT),
+    CREATE_ISCHEMA(MessageFactory.CREATE_ISCHEMA_EVENT),
+    ALTER_ISCHEMA(MessageFactory.ALTER_ISCHEMA_EVENT),
+    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);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 c59ca02..5976c48 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
@@ -59,6 +59,13 @@ public abstract class MessageFactory {
   public static final String ADD_UNIQUECONSTRAINT_EVENT = "ADD_UNIQUECONSTRAINT";
   public static final String ADD_NOTNULLCONSTRAINT_EVENT = "ADD_NOTNULLCONSTRAINT";
   public static final String DROP_CONSTRAINT_EVENT = "DROP_CONSTRAINT";
+  public static final String CREATE_ISCHEMA_EVENT = "CREATE_ISCHEMA";
+  public static final String ALTER_ISCHEMA_EVENT = "ALTER_ISCHEMA";
+  public static final String DROP_ISCHEMA_EVENT = "DROP_ISCHEMA";
+  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";
+
 
   private static MessageFactory instance = null;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MISchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MISchema.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MISchema.java
new file mode 100644
index 0000000..e64b0e9
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MISchema.java
@@ -0,0 +1,107 @@
+/*
+ * 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 MISchema {
+  private int schemaType;
+  private String name;
+  private MDatabase db;
+  private int compatibility;
+  private int validationLevel;
+  private boolean canEvolve;
+  private String schemaGroup;
+  private String description;
+
+  public MISchema(int schemaType, String name, MDatabase db, int compatibility,
+                  int validationLevel, boolean canEvolve, String schemaGroup, String description) {
+    this.schemaType = schemaType;
+    this.name = name;
+    this.db= db;
+    this.compatibility = compatibility;
+    this.validationLevel = validationLevel;
+    this.canEvolve = canEvolve;
+    this.schemaGroup = schemaGroup;
+    this.description = description;
+  }
+
+  public int getSchemaType() {
+    return schemaType;
+  }
+
+  public void setSchemaType(int schemaType) {
+    this.schemaType = schemaType;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public MDatabase getDb() {
+    return db;
+  }
+
+  public MISchema setDb(MDatabase db) {
+    this.db = db;
+    return this;
+  }
+
+  public int getCompatibility() {
+    return compatibility;
+  }
+
+  public void setCompatibility(int compatibility) {
+    this.compatibility = compatibility;
+  }
+
+  public int getValidationLevel() {
+    return validationLevel;
+  }
+
+  public void setValidationLevel(int validationLevel) {
+    this.validationLevel = validationLevel;
+  }
+
+  public boolean getCanEvolve() {
+    return canEvolve;
+  }
+
+  public void setCanEvolve(boolean canEvolve) {
+    this.canEvolve = canEvolve;
+  }
+
+  public String getSchemaGroup() {
+    return schemaGroup;
+  }
+
+  public void setSchemaGroup(String schemaGroup) {
+    this.schemaGroup = schemaGroup;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSchemaVersion.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSchemaVersion.java
new file mode 100644
index 0000000..7c8a6d4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSchemaVersion.java
@@ -0,0 +1,127 @@
+/*
+ * 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 MSchemaVersion {
+  private MISchema iSchema;
+  private int version;
+  private long createdAt;
+  private MColumnDescriptor cols;
+  private int state;
+  private String description;
+  private String schemaText;
+  private String fingerprint;
+  private String name;
+  private MSerDeInfo serDe;
+
+  public MSchemaVersion(MISchema iSchema, int version, long createdAt,
+                        MColumnDescriptor cols, int state, String description,
+                        String schemaText, String fingerprint, String name,
+                        MSerDeInfo serDe) {
+    this.iSchema = iSchema;
+    this.version = version;
+    this.createdAt = createdAt;
+    this.cols = cols;
+    this.state = state;
+    this.description = description;
+    this.schemaText = schemaText;
+    this.fingerprint = fingerprint;
+    this.name = name;
+    this.serDe = serDe;
+  }
+
+  public MISchema getiSchema() {
+    return iSchema;
+  }
+
+  public void setiSchema(MISchema iSchema) {
+    this.iSchema = iSchema;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  public long getCreatedAt() {
+    return createdAt;
+  }
+
+  public void setCreatedAt(long createdAt) {
+    this.createdAt = createdAt;
+  }
+
+  public MColumnDescriptor getCols() {
+    return cols;
+  }
+
+  public void setCols(MColumnDescriptor cols) {
+    this.cols = cols;
+  }
+
+  public int getState() {
+    return state;
+  }
+
+  public void setState(int state) {
+    this.state = state;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public String getSchemaText() {
+    return schemaText;
+  }
+
+  public void setSchemaText(String schemaText) {
+    this.schemaText = schemaText;
+  }
+
+  public String getFingerprint() {
+    return fingerprint;
+  }
+
+  public void setFingerprint(String fingerprint) {
+    this.fingerprint = fingerprint;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public MSerDeInfo getSerDe() {
+    return serDe;
+  }
+
+  public void setSerDe(MSerDeInfo serDe) {
+    this.serDe = serDe;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
index c4b27df..68f07e2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
@@ -24,16 +24,31 @@ public class MSerDeInfo {
   private String name;
   private String serializationLib;
   private Map<String, String> parameters;
+  private String description;
+  private String serializerClass;
+  private String deserializerClass;
+  private int serdeType;
 
   /**
+   *
    * @param name
    * @param serializationLib
    * @param parameters
+   * @param description
+   * @param serializerClass
+   * @param deserializerClass
+   * @param serdeType
    */
-  public MSerDeInfo(String name, String serializationLib, Map<String, String> parameters) {
+  public MSerDeInfo(String name, String serializationLib, Map<String, String> parameters,
+                    String description, String serializerClass, String deserializerClass,
+                    int serdeType) {
     this.name = name;
     this.serializationLib = serializationLib;
     this.parameters = parameters;
+    this.description = description;
+    this.serializerClass = serializerClass;
+    this.deserializerClass = deserializerClass;
+    this.serdeType = serdeType;
   }
 
   /**
@@ -78,4 +93,35 @@ public class MSerDeInfo {
     this.parameters = parameters;
   }
 
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public String getSerializerClass() {
+    return serializerClass;
+  }
+
+  public void setSerializerClass(String serializerClass) {
+    this.serializerClass = serializerClass;
+  }
+
+  public String getDeserializerClass() {
+    return deserializerClass;
+  }
+
+  public void setDeserializerClass(String deserializerClass) {
+    this.deserializerClass = deserializerClass;
+  }
+
+  public int getSerdeType() {
+    return serdeType;
+  }
+
+  public void setSerdeType(int serdeType) {
+    this.serdeType = serdeType;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 d343911..5aee2e3 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore.utils;
 
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.WMPoolSchedulingPolicy;
 
 import com.google.common.base.Predicates;

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/resources/datanucleus-log4j.properties
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/datanucleus-log4j.properties b/standalone-metastore/src/main/resources/datanucleus-log4j.properties
new file mode 100644
index 0000000..80f17e8
--- /dev/null
+++ b/standalone-metastore/src/main/resources/datanucleus-log4j.properties
@@ -0,0 +1,17 @@
+# Define the destination and format of our logging
+log4j.appender.A1=org.apache.log4j.FileAppender
+log4j.appender.A1.File=target/datanucleus.log
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%d{HH:mm:ss,SSS} (%t) %-5p [%c] - %m%n
+
+# DataNucleus Categories
+log4j.category.DataNucleus.JDO=INFO, A1
+log4j.category.DataNucleus.Cache=INFO, A1
+log4j.category.DataNucleus.MetaData=INFO, A1
+log4j.category.DataNucleus.General=INFO, A1
+log4j.category.DataNucleus.Transaction=INFO, A1
+log4j.category.DataNucleus.Datastore=DEBUG, A1
+log4j.category.DataNucleus.ValueGeneration=DEBUG, A1
+
+log4j.category.DataNucleus.Enhancer=INFO, A1
+log4j.category.DataNucleus.SchemaTool=INFO, A1

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 f408de5..7612509 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -270,6 +270,18 @@
            <column name="PARAM_VALUE" length="32672" jdbc-type="VARCHAR"/>
         </value>
       </field>
+      <field name="description">
+        <column name="DESCRIPTION" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
+      <field name="serializerClass">
+        <column name="SERIALIZER_CLASS" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
+      <field name="deserializerClass">
+        <column name="DESERIALIZER_CLASS" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
+      <field name="serdeType">
+        <column name="SERDE_TYPE" jdbc-type="integer" allows-null="true"/>
+      </field>
     </class>
 
     <class name="MOrder" embedded-only="true" table="SORT_ORDER" detachable="true">
@@ -1223,6 +1235,71 @@
       </index>
     </class>
 
+    <class name="MISchema" identity-type="datastore" table="I_SCHEMA" detachable="true">
+      <datastore-identity>
+        <column name="SCHEMA_ID"/>
+      </datastore-identity>
+      <field name="schemaType">
+        <column name="SCHEMA_TYPE" jdbc-type="integer"/>
+      </field>
+      <field name="name">
+        <column name="NAME" jdbc-type="varchar" length="256"/>
+      </field>
+      <field name="db">
+        <column name="DB_ID"/>
+      </field>
+      <field name="compatibility">
+        <column name="COMPATIBILITY" jdbc-type="integer"/>
+      </field>
+      <field name="validationLevel">
+        <column name="VALIDATION_LEVEL" jdbc-type="integer"/>
+      </field>
+      <field name="canEvolve">
+        <column name="CAN_EVOLVE"/>
+      </field>
+      <field name="schemaGroup">
+        <column name="SCHEMA_GROUP" jdbc-type="varchar" length="256" allows-null="true"/>
+      </field>
+      <field name="description">
+        <column name="DESCRIPTION" jdbc-type="varchar" length="4000" allows-null="true"/>
+      </field>
+    </class>
+
+    <class name="MSchemaVersion" identity-type="datastore" table="SCHEMA_VERSION" detachable="true">
+      <datastore-identity>
+        <column name="SCHEMA_VERSION_ID"/>
+      </datastore-identity>
+      <field name="iSchema">
+        <column name="SCHEMA_ID"/>
+      </field>
+      <field name="version">
+        <column name="VERSION" jdbc-type="integer"/>
+      </field>
+      <field name="createdAt">
+        <column name="CREATED_AT" jdbc-type="bigint"/>
+      </field>
+      <field name="cols">
+          <column name="CD_ID"/>
+      </field>
+      <field name="state">
+        <column name="STATE" jdbc-type="integer"/>
+      </field>
+      <field name="description">
+        <column name="DESCRIPTION" jdbc-type="varchar" length="4000" allows-null="true"/>
+      </field>
+      <field name="schemaText" default-fetch-group="false">
+        <column name="SCHEMA_TEXT" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="fingerprint">
+        <column name="FINGERPRINT" jdbc-type="varchar" length="256" allows-null="true"/>
+      </field>
+      <field name="name">
+        <column name="SCHEMA_VERSION_NAME" jdbc-type="varchar" length="256" allows-null="true"/>
+      </field>
+      <field name="serDe">
+        <column name="SERDE_ID"/>
+      </field>
+    </class>
   </package>
 </jdo>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 f6b9fdd..31d175e 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
@@ -42,7 +42,7 @@ CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCH
 
 CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT);
 
-CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000));
+CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER);
 
 CREATE TABLE "APP"."PART_PRIVS" ("PART_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_PRIV" VARCHAR(128));
 
@@ -544,6 +544,34 @@ CREATE TABLE NEXT_WRITE_ID (
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
+CREATE TABLE "APP"."I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" integer not null,
+  "NAME" varchar(256) unique,
+  "DB_ID" bigint references "APP"."DBS" ("DB_ID"),
+  "COMPATIBILITY" integer not null,
+  "VALIDATION_LEVEL" integer not null,
+  "CAN_EVOLVE" char(1) not null,
+  "SCHEMA_GROUP" varchar(256),
+  "DESCRIPTION" varchar(4000)
+);
+
+CREATE TABLE "APP"."SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "APP"."I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" integer not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "APP"."CDS" ("CD_ID"),
+  "STATE" integer not null,
+  "DESCRIPTION" varchar(4000),
+  "SCHEMA_TEXT" clob,
+  "FINGERPRINT" varchar(256),
+  "SCHEMA_VERSION_NAME" varchar(256),
+  "SERDE_ID" bigint references "APP"."SERDES" ("SERDE_ID")
+);
+
+CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SCHEMA_ID", "VERSION");
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 99aed9f..7ad5147 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
@@ -43,6 +43,40 @@ ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_PK" PRIMARY KEY ("MAPP
 ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+-- Upgrades for Schema Registry objects
+ALTER TABLE "APP"."SERDES" ADD COLUMN "DESCRIPTION" VARCHAR(4000);
+ALTER TABLE "APP"."SERDES" ADD COLUMN "SERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "APP"."SERDES" ADD COLUMN "DESERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "APP"."SERDES" ADD COLUMN "SERDE_TYPE" INTEGER;
+
+CREATE TABLE "APP"."I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" integer not null,
+  "NAME" varchar(256) unique,
+  "DB_ID" bigint references "APP"."DBS" ("DB_ID"),
+  "COMPATIBILITY" integer not null,
+  "VALIDATION_LEVEL" integer not null,
+  "CAN_EVOLVE" char(1) not null,
+  "SCHEMA_GROUP" varchar(256),
+  "DESCRIPTION" varchar(4000)
+);
+
+CREATE TABLE "APP"."SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "APP"."I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" integer not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "APP"."CDS" ("CD_ID"),
+  "STATE" integer not null,
+  "DESCRIPTION" varchar(4000),
+  "SCHEMA_TEXT" clob,
+  "FINGERPRINT" varchar(256),
+  "SCHEMA_VERSION_NAME" varchar(256),
+  "SERDE_ID" bigint references "APP"."SERDES" ("SERDE_ID")
+);
+
+CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SCHEMA_ID", "VERSION");
+
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 
 -- 048-HIVE-14498

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 0147958..eb71d82 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
@@ -312,7 +312,11 @@ CREATE TABLE SERDES
 (
     SERDE_ID bigint NOT NULL,
     "NAME" nvarchar(128) NULL,
-    SLIB nvarchar(4000) NULL
+    SLIB nvarchar(4000) NULL,
+    "DESCRIPTION" nvarchar(4000),
+    "SERIALIZER_CLASS" nvarchar(4000),
+    "DESERIALIZER_CLASS" nvarchar(4000),
+    "SERDE_TYPE" int
 );
 
 ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
@@ -1149,6 +1153,33 @@ CREATE TABLE NEXT_WRITE_ID (
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" int not null,
+  "NAME" nvarchar(256) unique,
+  "DB_ID" bigint references "DBS" ("DB_ID"),
+  "COMPATIBILITY" int not null,
+  "VALIDATION_LEVEL" int not null,
+  "CAN_EVOLVE" bit not null,
+  "SCHEMA_GROUP" nvarchar(256),
+  "DESCRIPTION" nvarchar(4000),
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" int not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "CDS" ("CD_ID"),
+  "STATE" int not null,
+  "DESCRIPTION" nvarchar(4000),
+  "SCHEMA_TEXT" varchar(max),
+  "FINGERPRINT" nvarchar(256),
+  "SCHEMA_VERSION_NAME" nvarchar(256),
+  "SERDE_ID" bigint references "SERDES" ("SERDE_ID"),
+  unique ("SCHEMA_ID", "VERSION")
+);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 3dda4ed..2dc0e9b 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
@@ -103,6 +103,39 @@ ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK1 FOREIGN KEY (RP_ID) REFEREN
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK2 FOREIGN KEY (POOL_ID) REFERENCES WM_POOL (POOL_ID);
 
+-- Upgrades for Schema Registry objects
+ALTER TABLE "SERDES" ADD "DESCRIPTION" nvarchar(4000);
+ALTER TABLE "SERDES" ADD "SERIALIZER_CLASS" nvarchar(4000);
+ALTER TABLE "SERDES" ADD "DESERIALIZER_CLASS" nvarchar(4000);
+ALTER TABLE "SERDES" ADD "SERDE_TYPE" int;
+
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" int not null,
+  "NAME" nvarchar(256) unique,
+  "DB_ID" bigint references "DBS" ("DB_ID"),
+  "COMPATIBILITY" int not null,
+  "VALIDATION_LEVEL" int not null,
+  "CAN_EVOLVE" bit not null,
+  "SCHEMA_GROUP" nvarchar(256),
+  "DESCRIPTION" nvarchar(4000),
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" int not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "CDS" ("CD_ID"),
+  "STATE" int not null,
+  "DESCRIPTION" nvarchar(4000),
+  "SCHEMA_TEXT" varchar(max),
+  "FINGERPRINT" nvarchar(256),
+  "SCHEMA_VERSION_NAME" nvarchar(256),
+  "SERDE_ID" bigint references "SERDES" ("SERDE_ID"),
+  unique ("SCHEMA_ID", "VERSION")
+);
+
 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/12041d39/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 01937ef..40680a4 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
@@ -436,6 +436,10 @@ CREATE TABLE IF NOT EXISTS `SERDES` (
   `SERDE_ID` bigint(20) NOT NULL,
   `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `SLIB` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DESCRIPTION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SERIALIZER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DESERIALIZER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SERDE_TYPE` integer,
   PRIMARY KEY (`SERDE_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
@@ -1084,6 +1088,38 @@ CREATE TABLE NEXT_WRITE_ID (
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
+CREATE TABLE `I_SCHEMA` (
+  `SCHEMA_ID` BIGINT PRIMARY KEY,
+  `SCHEMA_TYPE` INTEGER NOT NULL,
+  `NAME` VARCHAR(256),
+  `DB_ID` BIGINT,
+  `COMPATIBILITY` INTEGER NOT NULL,
+  `VALIDATION_LEVEL` INTEGER NOT NULL,
+  `CAN_EVOLVE` bit(1) NOT NULL,
+  `SCHEMA_GROUP` VARCHAR(256),
+  `DESCRIPTION` VARCHAR(4000),
+  FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`),
+  KEY `UNIQUE_NAME` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE `SCHEMA_VERSION` (
+  `SCHEMA_VERSION_ID` bigint primary key,
+  `SCHEMA_ID` BIGINT,
+  `VERSION` INTEGER NOT NULL,
+  `CREATED_AT` BIGINT NOT NULL,
+  `CD_ID` BIGINT, 
+  `STATE` INTEGER NOT NULL,
+  `DESCRIPTION` VARCHAR(4000),
+  `SCHEMA_TEXT` mediumtext,
+  `FINGERPRINT` VARCHAR(256),
+  `SCHEMA_VERSION_NAME` VARCHAR(256),
+  `SERDE_ID` bigint, 
+  FOREIGN KEY (`SCHEMA_ID`) REFERENCES `I_SCHEMA` (`SCHEMA_ID`),
+  FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`),
+  FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`),
+  KEY `UNIQUE_VERSION` (`SCHEMA_ID`, `VERSION`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 2d4b526..29b7635 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -88,6 +88,44 @@ CREATE TABLE IF NOT EXISTS WM_MAPPING
     CONSTRAINT `WM_MAPPING_FK2` FOREIGN KEY (`POOL_ID`) REFERENCES `WM_POOL` (`POOL_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
+-- Upgrades for Schema Registry objects
+ALTER TABLE `SERDES` ADD COLUMN `DESCRIPTION` VARCHAR(4000);
+ALTER TABLE `SERDES` ADD COLUMN `SERIALIZER_CLASS` VARCHAR(4000);
+ALTER TABLE `SERDES` ADD COLUMN `DESERIALIZER_CLASS` VARCHAR(4000);
+ALTER TABLE `SERDES` ADD COLUMN `SERDE_TYPE` INTEGER;
+
+CREATE TABLE `I_SCHEMA` (
+  `SCHEMA_ID` BIGINT PRIMARY KEY,
+  `SCHEMA_TYPE` INTEGER NOT NULL,
+  `NAME` VARCHAR(256),
+  `DB_ID` BIGINT,
+  `COMPATIBILITY` INTEGER NOT NULL,
+  `VALIDATION_LEVEL` INTEGER NOT NULL,
+  `CAN_EVOLVE` bit(1) NOT NULL,
+  `SCHEMA_GROUP` VARCHAR(256),
+  `DESCRIPTION` VARCHAR(4000),
+  FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`),
+  KEY `UNIQUE_NAME` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE `SCHEMA_VERSION` (
+  `SCHEMA_VERSION_ID` bigint primary key,
+  `SCHEMA_ID` BIGINT,
+  `VERSION` INTEGER NOT NULL,
+  `CREATED_AT` BIGINT NOT NULL,
+  `CD_ID` BIGINT, 
+  `STATE` INTEGER NOT NULL,
+  `DESCRIPTION` VARCHAR(4000),
+  `SCHEMA_TEXT` mediumtext,
+  `FINGERPRINT` VARCHAR(256),
+  `SCHEMA_VERSION_NAME` VARCHAR(256),
+  `SERDE_ID` bigint, 
+  FOREIGN KEY (`SCHEMA_ID`) REFERENCES `I_SCHEMA` (`SCHEMA_ID`),
+  FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`),
+  FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`),
+  KEY `UNIQUE_VERSION` (`SCHEMA_ID`, `VERSION`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
 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 ' ';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index fdc866d..e3410c8 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -100,7 +100,11 @@ CREATE TABLE SERDES
 (
     SERDE_ID NUMBER NOT NULL,
     "NAME" VARCHAR2(128) NULL,
-    SLIB VARCHAR2(4000) NULL
+    SLIB VARCHAR2(4000) NULL,
+    "DESCRIPTION" VARCHAR2(4000),
+    "SERIALIZER_CLASS" VARCHAR2(4000),
+    "DESERIALIZER_CLASS" VARCHAR2(4000),
+    "SERDE_TYPE" NUMBER
 );
 
 ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
@@ -1057,6 +1061,33 @@ CREATE TABLE NEXT_WRITE_ID (
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" number primary key,
+  "SCHEMA_TYPE" number not null,
+  "NAME" varchar2(256) unique,
+  "DB_ID" number references "DBS" ("DB_ID"),
+  "COMPATIBILITY" number not null,
+  "VALIDATION_LEVEL" number not null,
+  "CAN_EVOLVE" number(1) not null,
+  "SCHEMA_GROUP" varchar2(256),
+  "DESCRIPTION" varchar2(4000)
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" number primary key,
+  "SCHEMA_ID" number references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" number not null,
+  "CREATED_AT" number not null,
+  "CD_ID" number references "CDS" ("CD_ID"), 
+  "STATE" number not null,
+  "DESCRIPTION" varchar2(4000),
+  "SCHEMA_TEXT" clob,
+  "FINGERPRINT" varchar2(256),
+  "SCHEMA_VERSION_NAME" varchar2(256),
+  "SERDE_ID" number references "SERDES" ("SERDE_ID"), 
+  UNIQUE ("SCHEMA_ID", "VERSION")
+);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index 4e3f333..3320012 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -104,6 +104,40 @@ ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK1 FOREIGN KEY (RP_ID) REFEREN
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK2 FOREIGN KEY (POOL_ID) REFERENCES WM_POOL (POOL_ID);
 
+-- Upgrades for Schema Registry objects
+ALTER TABLE "SERDES" ADD "DESCRIPTION" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD "SERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD "DESERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD "SERDE_TYPE" INTEGER;
+
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" number primary key,
+  "SCHEMA_TYPE" number not null,
+  "NAME" varchar2(256) unique,
+  "DB_ID" number references "DBS" ("DB_ID"),
+  "COMPATIBILITY" number not null,
+  "VALIDATION_LEVEL" number not null,
+  "CAN_EVOLVE" number(1) not null,
+  "SCHEMA_GROUP" varchar2(256),
+  "DESCRIPTION" varchar2(4000)
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" number primary key,
+  "SCHEMA_ID" number references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" number not null,
+  "CREATED_AT" number not null,
+  "CD_ID" number references "CDS" ("CD_ID"), 
+  "STATE" number not null,
+  "DESCRIPTION" varchar2(4000),
+  "SCHEMA_TEXT" clob,
+  "FINGERPRINT" varchar2(256),
+  "SCHEMA_VERSION_NAME" varchar2(256),
+  "SERDE_ID" number references "SERDES" ("SERDE_ID"), 
+  UNIQUE ("SCHEMA_ID", "VERSION")
+);
+
+
 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 Status from dual;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 3cc2989..61dce6f 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -319,7 +319,11 @@ CREATE TABLE "SEQUENCE_TABLE" (
 CREATE TABLE "SERDES" (
     "SERDE_ID" bigint NOT NULL,
     "NAME" character varying(128) DEFAULT NULL::character varying,
-    "SLIB" character varying(4000) DEFAULT NULL::character varying
+    "SLIB" character varying(4000) DEFAULT NULL::character varying,
+    "DESCRIPTION" varchar(4000),
+    "SERIALIZER_CLASS" varchar(4000),
+    "DESERIALIZER_CLASS" varchar(4000),
+    "SERDE_TYPE" integer
 );
 
 
@@ -1749,6 +1753,33 @@ CREATE TABLE NEXT_WRITE_ID (
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" integer not null,
+  "NAME" varchar(256) unique,
+  "DB_ID" bigint references "DBS" ("DB_ID"),
+  "COMPATIBILITY" integer not null,
+  "VALIDATION_LEVEL" integer not null,
+  "CAN_EVOLVE" boolean not null,
+  "SCHEMA_GROUP" varchar(256),
+  "DESCRIPTION" varchar(4000)
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" integer not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "CDS" ("CD_ID"), 
+  "STATE" integer not null,
+  "DESCRIPTION" varchar(4000),
+  "SCHEMA_TEXT" text,
+  "FINGERPRINT" varchar(256),
+  "SCHEMA_VERSION_NAME" varchar(256),
+  "SERDE_ID" bigint references "SERDES" ("SERDE_ID"), 
+  unique ("SCHEMA_ID", "VERSION")
+);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index 83f71a0..89f43b7 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -118,6 +118,40 @@ ALTER TABLE ONLY "WM_MAPPING"
 ALTER TABLE ONLY "WM_MAPPING"
     ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "WM_POOL" ("POOL_ID") DEFERRABLE;
 
+-- Upgrades for Schema Registry objects
+ALTER TABLE "SERDES" ADD COLUMN "DESCRIPTION" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD COLUMN "SERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD COLUMN "DESERIALIZER_CLASS" VARCHAR(4000);
+ALTER TABLE "SERDES" ADD COLUMN "SERDE_TYPE" INTEGER;
+
+CREATE TABLE "I_SCHEMA" (
+  "SCHEMA_ID" bigint primary key,
+  "SCHEMA_TYPE" integer not null,
+  "NAME" varchar(256) unique,
+  "DB_ID" bigint references "DBS" ("DB_ID"),
+  "COMPATIBILITY" integer not null,
+  "VALIDATION_LEVEL" integer not null,
+  "CAN_EVOLVE" boolean not null,
+  "SCHEMA_GROUP" varchar(256),
+  "DESCRIPTION" varchar(4000)
+);
+
+CREATE TABLE "SCHEMA_VERSION" (
+  "SCHEMA_VERSION_ID" bigint primary key,
+  "SCHEMA_ID" bigint references "I_SCHEMA" ("SCHEMA_ID"),
+  "VERSION" integer not null,
+  "CREATED_AT" bigint not null,
+  "CD_ID" bigint references "CDS" ("CD_ID"), 
+  "STATE" integer not null,
+  "DESCRIPTION" varchar(4000),
+  "SCHEMA_TEXT" text,
+  "FINGERPRINT" varchar(256),
+  "SCHEMA_VERSION_NAME" varchar(256),
+  "SERDE_ID" bigint references "SERDES" ("SERDE_ID"), 
+  unique ("SCHEMA_ID", "VERSION")
+);
+
+
 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';