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/01/18 17:56:17 UTC

[57/70] [abbrv] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 53a8669..7afa456 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2782,7 +2782,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public void createOrDropTriggerToPoolMapping(String resourcePlanName, String triggerName,
       String poolPath, boolean shouldDrop) throws AlreadyExistsException, NoSuchObjectException,
       InvalidObjectException, MetaException, TException {
-    WMCreateOrDropTriggerToPoolMappingRequest request = new WMCreateOrDropTriggerToPoolMappingRequest();
+    WMCreateOrDropTriggerToPoolMappingRequest request =
+        new WMCreateOrDropTriggerToPoolMappingRequest();
     request.setResourcePlanName(resourcePlanName);
     request.setTriggerName(triggerName);
     request.setPoolPath(poolPath);
@@ -2790,4 +2791,74 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.create_or_drop_wm_trigger_to_pool_mapping(request);
   }
 
+  public void createISchema(ISchema schema) throws TException {
+    client.create_ischema(schema);
+  }
+
+  @Override
+  public void alterISchema(String schemaName, ISchema newSchema) throws TException {
+    client.alter_ischema(schemaName, newSchema);
+  }
+
+  @Override
+  public ISchema getISchema(String name) throws TException {
+    return client.get_ischema(name);
+  }
+
+  @Override
+  public void dropISchema(String name) throws TException {
+    client.drop_ischema(name);
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws TException {
+    client.add_schema_version(schemaVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(String schemaName, int version) throws TException {
+    return client.get_schema_version(schemaName, version);
+  }
+
+  @Override
+  public SchemaVersion getSchemaLatestVersion(String schemaName) throws TException {
+    return client.get_schema_latest_version(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaAllVersions(String schemaName) throws TException {
+    return client.get_schema_all_versions(schemaName);
+  }
+
+  @Override
+  public void dropSchemaVersion(String schemaName, int version) throws TException {
+    client.drop_schema_version(schemaName, version);
+  }
+
+  @Override
+  public FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst rqst) throws TException {
+    return client.get_schemas_by_cols(rqst);
+  }
+
+  @Override
+  public void mapSchemaVersionToSerde(String schemaName, int version, String serdeName)
+      throws TException {
+    client.map_schema_version_to_serde(schemaName, version, serdeName);
+  }
+
+  @Override
+  public void setSchemaVersionState(String schemaName, int version, SchemaVersionState state)
+      throws TException {
+    client.set_schema_version_state(schemaName, version, state);
+  }
+
+  @Override
+  public void addSerDe(SerDeInfo serDeInfo) throws TException {
+    client.add_serde(serDeInfo);
+  }
+
+  @Override
+  public SerDeInfo getSerDe(String serDeName) throws TException {
+    return client.get_serde(serDeName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 3261405..3c64c7e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsRqst;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
 import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
@@ -59,6 +61,7 @@ import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.ISchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -91,6 +94,9 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
@@ -1858,4 +1864,156 @@ public interface IMetaStoreClient {
   void createOrDropTriggerToPoolMapping(String resourcePlanName, String triggerName,
       String poolPath, boolean shouldDrop) throws AlreadyExistsException, NoSuchObjectException,
       InvalidObjectException, MetaException, TException;
+
+  /**
+   * Create a new schema.  This is really a schema container, as there will be specific versions
+   * of the schema that have columns, etc.
+   * @param schema schema to create
+   * @throws AlreadyExistsException if a schema of this name already exists
+   * @throws NoSuchObjectException database references by this schema does not exist
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void createISchema(ISchema schema) throws TException;
+
+  /**
+   * Alter an existing schema.
+   * @param schemaName name of the schema
+   * @param newSchema altered schema object
+   * @throws NoSuchObjectException no schema with this name could be found
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void alterISchema(String schemaName, ISchema newSchema) throws TException;
+
+  /**
+   * Fetch a schema.
+   * @param name name of the schema
+   * @return the schema or null if no such schema
+   * @throws NoSuchObjectException no schema matching this name exists
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  ISchema getISchema(String name) throws TException;
+
+  /**
+   * Drop an existing schema.  If there are schema versions of this, this call will fail.
+   * @param name name of the schema to drop
+   * @throws NoSuchObjectException no schema with this name could be found
+   * @throws InvalidOperationException attempt to drop a schema that has versions
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void dropISchema(String name) throws TException;
+
+  /**
+   * Add a new version to an existing schema.
+   * @param schemaVersion version object to add
+   * @throws AlreadyExistsException a version of this schema with the same version id already exists
+   * @throws NoSuchObjectException no schema with this name could be found
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void addSchemaVersion(SchemaVersion schemaVersion) throws TException;
+
+  /**
+   * Get a specific version of a schema.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @return the schema version or null if no such schema version
+   * @throws NoSuchObjectException no schema matching this name and version exists
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  SchemaVersion getSchemaVersion(String schemaName, int version) throws TException;
+
+  /**
+   * Get the latest version of a schema.
+   * @param schemaName name of the schema
+   * @return latest version of the schema or null if the schema does not exist or there are no
+   * version of the schema.
+   * @throws NoSuchObjectException no versions of schema matching this name exist
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  SchemaVersion getSchemaLatestVersion(String schemaName) throws TException;
+
+  /**
+   * Get all the extant versions of a schema.
+   * @param schemaName name of the schema.
+   * @return list of all the schema versions or null if this schema does not exist or has no
+   * versions.
+   * @throws NoSuchObjectException no versions of schema matching this name exist
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  List<SchemaVersion> getSchemaAllVersions(String schemaName) throws TException;
+
+  /**
+   * Drop a version of a schema.  Given that versions are supposed to be immutable you should
+   * think really hard before you call this method.  It should only be used for schema versions
+   * that were added in error and never referenced any data.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @throws NoSuchObjectException no matching version of the schema could be found
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void dropSchemaVersion(String schemaName, int version) throws TException;
+
+  /**
+   * Find all schema versions that have columns that match a query.
+   * @param rqst query, this can include column names, namespaces (actually stored in the
+   *             description field in FieldSchema), and types.
+   * @return The (possibly empty) list of schema name/version pairs that match.
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst rqst) throws TException;
+
+  /**
+   * Map a schema version to a serde.  This mapping is one-to-one, thus this will destroy any
+   * previous mappings for this schema version.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @param serdeName name of the serde
+   * @throws NoSuchObjectException no matching version of the schema could be found or no serde
+   * of the provided name could be found
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void mapSchemaVersionToSerde(String schemaName, int version, String serdeName) throws TException;
+
+  /**
+   * Set the state of a schema version.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @param state state to set the schema too
+   * @throws NoSuchObjectException no matching version of the schema could be found
+   * @throws InvalidOperationException attempt to make a state change that is not valid
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void setSchemaVersionState(String schemaName, int version, SchemaVersionState state) throws TException;
+
+  /**
+   * Add a serde.  This is primarily intended for use with SchemaRegistry objects, since serdes
+   * are automatically added when needed as part of creating and altering tables and partitions.
+   * @param serDeInfo serde to add
+   * @throws AlreadyExistsException serde of this name already exists
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  void addSerDe(SerDeInfo serDeInfo) throws TException;
+
+  /**
+   * Fetch a serde.  This is primarily intended for use with SchemaRegistry objects, since serdes
+   * are automatically fetched along with other information for tables and partitions.
+   * @param serDeName name of the serde
+   * @return the serde.
+   * @throws NoSuchObjectException no serde with this name exists.
+   * @throws MetaException general metastore error
+   * @throws TException general thrift error
+   */
+  SerDeInfo getSerDe(String serDeName) throws TException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index 3a351da..ed60bd4 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -27,21 +27,27 @@ import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
 import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AlterDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.DropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.InsertEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
@@ -220,6 +226,26 @@ public abstract class MetaStoreEventListener implements Configurable {
   public void onDropConstraint(DropConstraintEvent dropConstraintEvent) throws MetaException {
   }
 
+  public void onCreateISchema(CreateISchemaEvent createISchemaEvent) throws MetaException {
+  }
+
+  public void onAlterISchema(AlterISchemaEvent alterISchemaEvent) throws MetaException {
+  }
+
+  public void onDropISchema(DropISchemaEvent dropISchemaEvent) throws MetaException {
+  }
+
+  public void onAddSchemaVersion(AddSchemaVersionEvent addSchemaVersionEvent) throws MetaException {
+  }
+
+  public void onAlterSchemaVersion(AlterSchemaVersionEvent alterSchemaVersionEvent)
+      throws MetaException {
+  }
+
+  public void onDropSchemaVersion(DropSchemaVersionEvent dropSchemaVersionEvent)
+      throws MetaException {
+  }
+
   @Override
   public Configuration getConf() {
     return this.conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index 3899e3c..97a74d3 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -29,18 +29,24 @@ import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AlterDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.DropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.InsertEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
@@ -179,6 +185,42 @@ public class MetaStoreListenerNotifier {
               listener.onAddNotNullConstraint((AddNotNullConstraintEvent)event);
             }
           })
+          .put(EventType.CREATE_ISCHEMA, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onCreateISchema((CreateISchemaEvent)event);
+            }
+          })
+          .put(EventType.ALTER_ISCHEMA, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAlterISchema((AlterISchemaEvent)event);
+            }
+          })
+          .put(EventType.DROP_ISCHEMA, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropISchema((DropISchemaEvent)event);
+            }
+          })
+          .put(EventType.ADD_SCHEMA_VERSION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddSchemaVersion((AddSchemaVersionEvent) event);
+            }
+          })
+          .put(EventType.ALTER_SCHEMA_VERSION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAlterSchemaVersion((AlterSchemaVersionEvent) event);
+            }
+          })
+          .put(EventType.DROP_SCHEMA_VERSION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropSchemaVersion((DropSchemaVersionEvent) event);
+            }
+          })
           .build()
   );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 3a1bc13..6984ef9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -93,6 +93,7 @@ import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.ISchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -122,7 +123,13 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaCompatibility;
+import org.apache.hadoop.hive.metastore.api.SchemaType;
+import org.apache.hadoop.hive.metastore.api.SchemaValidation;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SerdeType;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -152,6 +159,7 @@ import org.apache.hadoop.hive.metastore.model.MDelegationToken;
 import org.apache.hadoop.hive.metastore.model.MFieldSchema;
 import org.apache.hadoop.hive.metastore.model.MFunction;
 import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
+import org.apache.hadoop.hive.metastore.model.MISchema;
 import org.apache.hadoop.hive.metastore.model.MIndex;
 import org.apache.hadoop.hive.metastore.model.MMasterKey;
 import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
@@ -166,6 +174,7 @@ import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
 import org.apache.hadoop.hive.metastore.model.MResourceUri;
 import org.apache.hadoop.hive.metastore.model.MRole;
 import org.apache.hadoop.hive.metastore.model.MRoleMap;
+import org.apache.hadoop.hive.metastore.model.MSchemaVersion;
 import org.apache.hadoop.hive.metastore.model.MSerDeInfo;
 import org.apache.hadoop.hive.metastore.model.MStorageDescriptor;
 import org.apache.hadoop.hive.metastore.model.MStringList;
@@ -1725,15 +1734,22 @@ public class ObjectStore implements RawStore, Configurable {
     if (ms == null) {
       throw new MetaException("Invalid SerDeInfo object");
     }
-    return new SerDeInfo(ms.getName(), ms.getSerializationLib(), convertMap(ms.getParameters()));
+    SerDeInfo serde =
+        new SerDeInfo(ms.getName(), ms.getSerializationLib(), convertMap(ms.getParameters()));
+    if (ms.getDescription() != null) serde.setDescription(ms.getDescription());
+    if (ms.getSerializerClass() != null) serde.setSerializerClass(ms.getSerializerClass());
+    if (ms.getDeserializerClass() != null) serde.setDeserializerClass(ms.getDeserializerClass());
+    if (ms.getSerdeType() > 0) serde.setSerdeType(SerdeType.findByValue(ms.getSerdeType()));
+    return serde;
   }
 
   private MSerDeInfo convertToMSerDeInfo(SerDeInfo ms) throws MetaException {
     if (ms == null) {
       throw new MetaException("Invalid SerDeInfo object");
     }
-    return new MSerDeInfo(ms.getName(), ms.getSerializationLib(), ms
-        .getParameters());
+    return new MSerDeInfo(ms.getName(), ms.getSerializationLib(), ms.getParameters(),
+        ms.getDescription(), ms.getSerializerClass(), ms.getDeserializerClass(),
+        ms.getSerdeType() == null ? 0 : ms.getSerdeType().getValue());
   }
 
   /**
@@ -9591,6 +9607,392 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
+  @Override
+  public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
+      NoSuchObjectException {
+    boolean committed = false;
+    MISchema mSchema = convertToMISchema(schema);
+    try {
+      openTransaction();
+      if (getMISchema(schema.getName()) != null) {
+        throw new AlreadyExistsException("Schema with name " + schema.getName() + " already exists");
+      }
+      pm.makePersistent(mSchema);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public void alterISchema(String schemaName, ISchema newSchema)
+      throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MISchema oldMSchema = getMISchema(schemaName);
+      if (oldMSchema == null) {
+        throw new NoSuchObjectException("Schema " + schemaName + " does not exist");
+      }
+
+      // Don't support changing name or type
+      oldMSchema.setCompatibility(newSchema.getCompatibility().getValue());
+      oldMSchema.setValidationLevel(newSchema.getValidationLevel().getValue());
+      oldMSchema.setCanEvolve(newSchema.isCanEvolve());
+      if (newSchema.isSetSchemaGroup()) oldMSchema.setSchemaGroup(newSchema.getSchemaGroup());
+      if (newSchema.isSetDescription()) oldMSchema.setDescription(newSchema.getDescription());
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public ISchema getISchema(String schemaName) throws MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      ISchema schema = convertToISchema(getMISchema(schemaName));
+      committed = commitTransaction();
+      return schema;
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  private MISchema getMISchema(String schemaName) {
+    Query query = null;
+    try {
+      schemaName = normalizeIdentifier(schemaName);
+      query = pm.newQuery(MISchema.class, "name == schemaName");
+      query.declareParameters("java.lang.String schemaName");
+      query.setUnique(true);
+      MISchema mSchema = (MISchema)query.execute(schemaName);
+      pm.retrieve(mSchema);
+      return mSchema;
+    } finally {
+      if (query != null) query.closeAll();
+    }
+  }
+
+  @Override
+  public void dropISchema(String schemaName) throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MISchema mSchema = getMISchema(schemaName);
+      if (mSchema != null) {
+        pm.deletePersistentAll(mSchema);
+      } else {
+        throw new NoSuchObjectException("Schema " + schemaName + " does not exist");
+      }
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion)
+      throws AlreadyExistsException, NoSuchObjectException, MetaException {
+    boolean committed = false;
+    MSchemaVersion mSchemaVersion = convertToMSchemaVersion(schemaVersion);
+    try {
+      openTransaction();
+      // Make sure it doesn't already exist
+      if (getMSchemaVersion(schemaVersion.getSchemaName(), schemaVersion.getVersion()) != null) {
+        throw new AlreadyExistsException("Schema name " + schemaVersion.getSchemaName() +
+            " version " + schemaVersion.getVersion() + " already exists");
+      }
+      // Make sure the referenced Schema exists
+      if (getMISchema(schemaVersion.getSchemaName()) == null) {
+        throw new NoSuchObjectException("Schema " + schemaVersion.getSchemaName() + " does not exist");
+      }
+      pm.makePersistent(mSchemaVersion);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();;
+    }
+  }
+
+  @Override
+  public void alterSchemaVersion(String schemaName, int version, SchemaVersion newVersion)
+      throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MSchemaVersion oldMSchemaVersion = getMSchemaVersion(schemaName, version);
+      if (oldMSchemaVersion == null) {
+        throw new NoSuchObjectException("No schema of name " + schemaName + " with version " +
+            version + " exists");
+      }
+
+      // We only support changing the SerDe mapping and the state.
+      if (newVersion.isSetSerDe()) oldMSchemaVersion.setSerDe(convertToMSerDeInfo(newVersion.getSerDe()));
+      if (newVersion.isSetState()) oldMSchemaVersion.setState(newVersion.getState().getValue());
+      committed = commitTransaction();
+    } finally {
+      if (!committed) commitTransaction();
+    }
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(String schemaName, int version) throws MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      SchemaVersion schemaVersion = convertToSchemaVersion(getMSchemaVersion(schemaName, version));
+      committed = commitTransaction();
+      return schemaVersion;
+    } finally {
+      if (!committed) rollbackTransaction();;
+    }
+  }
+
+  private MSchemaVersion getMSchemaVersion(String schemaName, int version) {
+    Query query = null;
+    try {
+      schemaName = normalizeIdentifier(schemaName);
+      query = pm.newQuery(MSchemaVersion.class, "iSchema.name == schemaName && version == schemaVersion");
+      query.declareParameters("java.lang.String schemaName, java.lang.Integer schemaVersion");
+      query.setUnique(true);
+      MSchemaVersion mSchemaVersion = (MSchemaVersion)query.execute(schemaName, version);
+      pm.retrieve(mSchemaVersion);
+      if (mSchemaVersion != null) {
+        pm.retrieveAll(mSchemaVersion.getCols());
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
+      }
+      return mSchemaVersion;
+    } finally {
+      if (query != null) query.closeAll();
+    }
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(String schemaName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      schemaName = normalizeIdentifier(schemaName);
+      query = pm.newQuery(MSchemaVersion.class, "iSchema.name == schemaName");
+      query.declareParameters("java.lang.String schemaName");
+      query.setUnique(true);
+      query.setOrdering("version descending");
+      query.setRange(0, 1);
+      MSchemaVersion mSchemaVersion = (MSchemaVersion)query.execute(schemaName);
+      pm.retrieve(mSchemaVersion);
+      if (mSchemaVersion != null) {
+        pm.retrieveAll(mSchemaVersion.getCols());
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
+      }
+      committed = commitTransaction();
+      return mSchemaVersion == null ? null : convertToSchemaVersion(mSchemaVersion);
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(String schemaName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      schemaName = normalizeIdentifier(schemaName);
+      query = pm.newQuery(MSchemaVersion.class, "iSchema.name == schemaName");
+      query.declareParameters("java.lang.String schemaName");
+      query.setOrdering("version descending");
+      List<MSchemaVersion> mSchemaVersions = query.setParameters(schemaName).executeList();
+      pm.retrieveAll(mSchemaVersions);
+      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) return null;
+      List<SchemaVersion> schemaVersions = new ArrayList<>(mSchemaVersions.size());
+      for (MSchemaVersion mSchemaVersion : mSchemaVersions) {
+        pm.retrieveAll(mSchemaVersion.getCols());
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
+        schemaVersions.add(convertToSchemaVersion(mSchemaVersion));
+      }
+      committed = commitTransaction();
+      return schemaVersions;
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace,
+                                                        String type) throws MetaException {
+    if (colName == null && colNamespace == null) {
+      // Don't allow a query that returns everything, it will blow stuff up.
+      throw new MetaException("You must specify column name or column namespace, else your query " +
+          "may be too large");
+    }
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      if (colName != null) colName = normalizeIdentifier(colName);
+      if (type != null) type = normalizeIdentifier(type);
+      Map<String, String> parameters = new HashMap<>(3);
+      StringBuilder sql = new StringBuilder("select SCHEMA_VERSION_ID from " +
+          "SCHEMA_VERSION, COLUMNS_V2 where SCHEMA_VERSION.CD_ID = COLUMNS_V2.CD_ID ");
+      if (colName != null) {
+        sql.append("and COLUMNS_V2.COLUMN_NAME = :colName ");
+        parameters.put("colName", colName);
+      }
+      if (colNamespace != null) {
+        sql.append("and COLUMNS_V2.COMMENT = :colComment ");
+        parameters.put("colComment", colNamespace);
+      }
+      if (type != null) {
+        sql.append("and COLUMNS_V2.TYPE_NAME = :colType ");
+        parameters.put("colType", type);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getSchemaVersionsByColumns going to execute query " + sql.toString());
+        LOG.debug("With parameters");
+        for (Map.Entry<String, String> p : parameters.entrySet()) {
+          LOG.debug(p.getKey() + " : " + p.getValue());
+        }
+      }
+      query = pm.newQuery("javax.jdo.query.SQL", sql.toString());
+      query.setClass(MSchemaVersion.class);
+      List<MSchemaVersion> mSchemaVersions = query.setNamedParameters(parameters).executeList();
+      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) return Collections.emptyList();
+      pm.retrieveAll(mSchemaVersions);
+      List<SchemaVersion> schemaVersions = new ArrayList<>(mSchemaVersions.size());
+      for (MSchemaVersion mSchemaVersion : mSchemaVersions) {
+        pm.retrieveAll(mSchemaVersion.getCols());
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
+        schemaVersions.add(convertToSchemaVersion(mSchemaVersion));
+      }
+      committed = commitTransaction();
+      return schemaVersions;
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+
+  }
+
+  @Override
+  public void dropSchemaVersion(String schemaName, int version) throws NoSuchObjectException,
+      MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MSchemaVersion mSchemaVersion = getMSchemaVersion(schemaName, version);
+      if (mSchemaVersion != null) {
+        pm.deletePersistentAll(mSchemaVersion);
+      } else {
+        throw new NoSuchObjectException("Schema " + schemaName + " of version " + version +
+            "does not exist");
+      }
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public SerDeInfo getSerDeInfo(String serDeName) throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MSerDeInfo mSerDeInfo = getMSerDeInfo(serDeName);
+      if (mSerDeInfo == null) {
+        throw new NoSuchObjectException("No SerDe named " + serDeName);
+      }
+      SerDeInfo serde = convertToSerDeInfo(mSerDeInfo);
+      committed = commitTransaction();
+      return serde;
+    } finally {
+      if (!committed) rollbackTransaction();;
+    }
+  }
+
+  private MSerDeInfo getMSerDeInfo(String serDeName) throws MetaException {
+    Query query = null;
+    try {
+      query = pm.newQuery(MSerDeInfo.class, "name == serDeName");
+      query.declareParameters("java.lang.String serDeName");
+      query.setUnique(true);
+      MSerDeInfo mSerDeInfo = (MSerDeInfo)query.execute(serDeName);
+      pm.retrieve(mSerDeInfo);
+      return mSerDeInfo;
+    } finally {
+      if (query != null) query.closeAll();
+    }
+  }
+
+  @Override
+  public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      if (getMSerDeInfo(serde.getName()) != null) {
+        throw new AlreadyExistsException("Serde with name " + serde.getName() + " already exists");
+      }
+      MSerDeInfo mSerde = convertToMSerDeInfo(serde);
+      pm.makePersistent(mSerde);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+
+  }
+
+  private MISchema convertToMISchema(ISchema schema) throws NoSuchObjectException {
+    return new MISchema(schema.getSchemaType().getValue(),
+                        normalizeIdentifier(schema.getName()),
+                        getMDatabase(schema.getDbName()),
+                        schema.getCompatibility().getValue(),
+                        schema.getValidationLevel().getValue(),
+                        schema.isCanEvolve(),
+                        schema.isSetSchemaGroup() ? schema.getSchemaGroup() : null,
+                        schema.isSetDescription() ? schema.getDescription() : null);
+  }
+
+  private ISchema convertToISchema(MISchema mSchema) {
+    if (mSchema == null) return null;
+    ISchema schema = new ISchema(SchemaType.findByValue(mSchema.getSchemaType()),
+                                 mSchema.getName(),
+                                 mSchema.getDb().getName(),
+                                 SchemaCompatibility.findByValue(mSchema.getCompatibility()),
+                                 SchemaValidation.findByValue(mSchema.getValidationLevel()),
+                                 mSchema.getCanEvolve());
+    if (mSchema.getDescription() != null) schema.setDescription(mSchema.getDescription());
+    if (mSchema.getSchemaGroup() != null) schema.setSchemaGroup(mSchema.getSchemaGroup());
+    return schema;
+  }
+
+  private MSchemaVersion convertToMSchemaVersion(SchemaVersion schemaVersion) throws MetaException {
+    return new MSchemaVersion(getMISchema(normalizeIdentifier(schemaVersion.getSchemaName())),
+                              schemaVersion.getVersion(),
+                              schemaVersion.getCreatedAt(),
+                              createNewMColumnDescriptor(convertToMFieldSchemas(schemaVersion.getCols())),
+                              schemaVersion.isSetState() ? schemaVersion.getState().getValue() : 0,
+                              schemaVersion.isSetDescription() ? schemaVersion.getDescription() : null,
+                              schemaVersion.isSetSchemaText() ? schemaVersion.getSchemaText() : null,
+                              schemaVersion.isSetFingerprint() ? schemaVersion.getFingerprint() : null,
+                              schemaVersion.isSetName() ? schemaVersion.getName() : null,
+                              schemaVersion.isSetSerDe() ? convertToMSerDeInfo(schemaVersion.getSerDe()) : null);
+  }
+
+  private SchemaVersion convertToSchemaVersion(MSchemaVersion mSchemaVersion) throws MetaException {
+    if (mSchemaVersion == null) return null;
+    SchemaVersion schemaVersion = new SchemaVersion(mSchemaVersion.getiSchema().getName(),
+                                                    mSchemaVersion.getVersion(),
+                                                    mSchemaVersion.getCreatedAt(),
+                                                    convertToFieldSchemas(mSchemaVersion.getCols().getCols()));
+    if (mSchemaVersion.getState() > 0) schemaVersion.setState(SchemaVersionState.findByValue(mSchemaVersion.getState()));
+    if (mSchemaVersion.getDescription() != null) schemaVersion.setDescription(mSchemaVersion.getDescription());
+    if (mSchemaVersion.getSchemaText() != null) schemaVersion.setSchemaText(mSchemaVersion.getSchemaText());
+    if (mSchemaVersion.getFingerprint() != null) schemaVersion.setFingerprint(mSchemaVersion.getFingerprint());
+    if (mSchemaVersion.getName() != null) schemaVersion.setName(mSchemaVersion.getName());
+    if (mSchemaVersion.getSerDe() != null) schemaVersion.setSerDe(convertToSerDeInfo(mSchemaVersion.getSerDe()));
+    return schemaVersion;
+  }
+
   /**
    * This is a cleanup method which is used to rollback a active transaction
    * if the success flag is false and close the associated Query object. This method is used

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index e6c7a58..8250485 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.ISchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -66,6 +67,8 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -810,4 +813,136 @@ public interface RawStore extends Configurable {
 
   void dropWMTriggerToPoolMapping(String resourcePlanName, String triggerName, String poolPath)
       throws NoSuchObjectException, InvalidOperationException, MetaException;
+
+  /**
+   * Create a new ISchema.
+   * @param schema schema to create
+   * @throws AlreadyExistsException there's already a schema with this name
+   * @throws MetaException general database exception
+   */
+  void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
+      NoSuchObjectException;
+
+  /**
+   * Alter an existing ISchema.  This assumes the caller has already checked that such a schema
+   * exists.
+   * @param schemaName name of the schema
+   * @param newSchema new schema object
+   * @throws NoSuchObjectException no function with this name exists
+   * @throws MetaException general database exception
+   */
+  void alterISchema(String schemaName, ISchema newSchema) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get an ISchema by name.
+   * @param schemaName name of the schema
+   * @return ISchema
+   * @throws MetaException general database exception
+   */
+  ISchema getISchema(String schemaName) throws MetaException;
+
+  /**
+   * Drop an ISchema.  This does not check whether there are valid versions of the schema in
+   * existence, it assumes the caller has already done that.
+   * @param schemaName name of the schema to drop
+   * @throws NoSuchObjectException no schema of this name exists
+   * @throws MetaException general database exception
+   */
+  void dropISchema(String schemaName) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Create a new version of an existing schema.
+   * @param schemaVersion version number
+   * @throws AlreadyExistsException a version of the schema with the same version number already
+   * exists.
+   * @throws InvalidObjectException the passed in SchemaVersion object has problems.
+   * @throws NoSuchObjectException no schema with the passed in name exists.
+   * @throws MetaException general database exception
+   */
+  void addSchemaVersion(SchemaVersion schemaVersion)
+      throws AlreadyExistsException, InvalidObjectException, NoSuchObjectException, MetaException;
+
+  /**
+   * Alter a schema version.  Note that the Thrift interface only supports changing the serde
+   * mapping and states.  This method does not guarantee it will check anymore than that.  This
+   * method does not understand the state transitions and just assumes that the new state it is
+   * passed is reasonable.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @param newVersion altered SchemaVersion
+   * @throws NoSuchObjectException no such version of the named schema exists
+   * @throws MetaException general database exception
+   */
+  void alterSchemaVersion(String schemaName, int version, SchemaVersion newVersion)
+      throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get a specific schema version.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @return the SchemaVersion
+   * @throws MetaException general database exception
+   */
+  SchemaVersion getSchemaVersion(String schemaName, int version) throws MetaException;
+
+  /**
+   * Get the latest version of a schema.
+   * @param schemaName name of the schema
+   * @return latest version of the schema
+   * @throws MetaException general database exception
+   */
+  SchemaVersion getLatestSchemaVersion(String schemaName) throws MetaException;
+
+  /**
+   * Get all of the versions of a schema
+   * @param schemaName name of the schema
+   * @return all versions of the schema
+   * @throws MetaException general database exception
+   */
+  List<SchemaVersion> getAllSchemaVersion(String schemaName) throws MetaException;
+
+  /**
+   * Find all SchemaVersion objects that match a query.  The query will select all SchemaVersions
+   * that are equal to all of the non-null passed in arguments.  That is, if arguments
+   * colName='name', colNamespace=null, type='string' are passed in, then all schemas that have
+   * a column with colName 'name' and type 'string' will be returned.
+   * @param colName column name.  Null is ok, which will cause this field to not be used in the
+   *                query.
+   * @param colNamespace column namespace.   Null is ok, which will cause this field to not be
+   *                     used in the query.
+   * @param type column type.   Null is ok, which will cause this field to not be used in the
+   *             query.
+   * @return List of all SchemaVersions that match.  Note that there is no expectation that these
+   * SchemaVersions derive from the same ISchema.  The list will be empty if there are no
+   * matching SchemaVersions.
+   * @throws MetaException general database exception
+   */
+  List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace, String type)
+      throws MetaException;
+
+  /**
+   * Drop a version of the schema.
+   * @param schemaName name of the schema
+   * @param version version of the schema
+   * @throws NoSuchObjectException no such version of the named schema exists
+   * @throws MetaException general database exception
+   */
+  void dropSchemaVersion(String schemaName, int version) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get serde information
+   * @param serDeName name of the SerDe
+   * @return the SerDe, or null if there is no such serde
+   * @throws NoSuchObjectException no serde with this name exists
+   * @throws MetaException general database exception
+   */
+  SerDeInfo getSerDeInfo(String serDeName) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Add a serde
+   * @param serde serde to add
+   * @throws AlreadyExistsException a serde of this name already exists
+   * @throws MetaException general database exception
+   */
+  void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 72c4a26..4d56a90 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.ISchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -82,6 +83,8 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
@@ -2258,6 +2261,78 @@ public class CachedStore implements RawStore, Configurable {
     return rawStore.getColStatsForTablePartitions(dbName, tableName);
   }
 
+  // TODO - not clear if we should cache these or not.  For now, don't bother
+  @Override
+  public void createISchema(ISchema schema)
+      throws AlreadyExistsException, NoSuchObjectException, MetaException {
+    rawStore.createISchema(schema);
+  }
+
+  @Override
+  public void alterISchema(String schemaName, ISchema newSchema)
+      throws NoSuchObjectException, MetaException {
+    rawStore.alterISchema(schemaName, newSchema);
+  }
+
+  @Override
+  public ISchema getISchema(String schemaName) throws MetaException {
+    return rawStore.getISchema(schemaName);
+  }
+
+  @Override
+  public void dropISchema(String schemaName) throws NoSuchObjectException, MetaException {
+    rawStore.dropISchema(schemaName);
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws
+      AlreadyExistsException, InvalidObjectException, NoSuchObjectException, MetaException {
+    rawStore.addSchemaVersion(schemaVersion);
+  }
+
+  @Override
+  public void alterSchemaVersion(String schemaName, int version, SchemaVersion newVersion) throws
+      NoSuchObjectException, MetaException {
+    rawStore.alterSchemaVersion(schemaName, version, newVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(String schemaName, int version) throws MetaException {
+    return rawStore.getSchemaVersion(schemaName, version);
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(String schemaName) throws MetaException {
+    return rawStore.getLatestSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(String schemaName) throws MetaException {
+    return rawStore.getAllSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace,
+                                                        String type) throws MetaException {
+    return rawStore.getSchemaVersionsByColumns(colName, colNamespace, type);
+  }
+
+  @Override
+  public void dropSchemaVersion(String schemaName, int version) throws NoSuchObjectException,
+      MetaException {
+    rawStore.dropSchemaVersion(schemaName, version);
+  }
+
+  @Override
+  public SerDeInfo getSerDeInfo(String serDeName) throws NoSuchObjectException, MetaException {
+    return rawStore.getSerDeInfo(serDeName);
+  }
+
+  @Override
+  public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
+    rawStore.addSerde(serde);
+  }
+
   public RawStore getRawStore() {
     return rawStore;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
index 7627d89..01693ec 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
@@ -73,7 +73,7 @@ public class DatabaseBuilder {
     return this;
   }
 
-  public Database build() throws TException {
+  public Database build() throws MetaException {
     if (name == null) throw new MetaException("You must name the database");
     Database db = new Database(name, description, location, params);
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
new file mode 100644
index 0000000..77adfbb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
@@ -0,0 +1,93 @@
+/*
+ * 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.MetaException;
+import org.apache.hadoop.hive.metastore.api.SchemaCompatibility;
+import org.apache.hadoop.hive.metastore.api.SchemaType;
+import org.apache.hadoop.hive.metastore.api.SchemaValidation;
+
+public class ISchemaBuilder {
+  private SchemaType schemaType; // required
+  private String name; // required
+  private String dbName; // required
+  private SchemaCompatibility compatibility; // required
+  private SchemaValidation validationLevel; // required
+  private boolean canEvolve; // required
+  private String schemaGroup; // optional
+  private String description; // optional
+
+  public ISchemaBuilder() {
+    compatibility = SchemaCompatibility.BACKWARD;
+    validationLevel = SchemaValidation.ALL;
+    canEvolve = true;
+    dbName = "default";
+  }
+
+  public ISchemaBuilder setSchemaType(SchemaType schemaType) {
+    this.schemaType = schemaType;
+    return this;
+  }
+
+  public ISchemaBuilder setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public ISchemaBuilder setDbName(String dbName) {
+    this.dbName = dbName;
+    return this;
+  }
+
+  public ISchemaBuilder setCompatibility(SchemaCompatibility compatibility) {
+    this.compatibility = compatibility;
+    return this;
+  }
+
+  public ISchemaBuilder setValidationLevel(SchemaValidation validationLevel) {
+    this.validationLevel = validationLevel;
+    return this;
+  }
+
+  public ISchemaBuilder setCanEvolve(boolean canEvolve) {
+    this.canEvolve = canEvolve;
+    return this;
+  }
+
+  public ISchemaBuilder setSchemaGroup(String schemaGroup) {
+    this.schemaGroup = schemaGroup;
+    return this;
+  }
+
+  public ISchemaBuilder setDescription(String description) {
+    this.description = description;
+    return this;
+  }
+
+  public ISchema build() throws MetaException {
+    if (schemaType == null || name == null) {
+      throw new MetaException("You must provide a schemaType and name");
+    }
+    ISchema iSchema =
+        new ISchema(schemaType, name, dbName, compatibility, validationLevel, canEvolve);
+    if (schemaGroup != null) iSchema.setSchemaGroup(schemaGroup);
+    if (description != null) iSchema.setDescription(description);
+    return iSchema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b6230001/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..289ef0a
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * 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.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; // 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;
+    super.setChild(this);
+  }
+
+  public SchemaVersionBuilder setSchemaName(String schemaName) {
+    this.schemaName = schemaName;
+    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 {
+    SchemaVersion schemaVersion = new SchemaVersion(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/b6230001/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/b6230001/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/b6230001/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/b6230001/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/b6230001/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/b6230001/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/b6230001/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/b6230001/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/b6230001/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;
+  }
+}